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/09 02:34:07 UTC

[GitHub] [iceberg] zhangjun0x01 opened a new pull request #1893: Flink : add filter push down for IcebergTableSource

zhangjun0x01 opened a new pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893


   add filter push down for IcebergTableSource


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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553166082



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +95,396 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
     Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, mixedResult.size());
+    assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    assertTrue("explain should contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id = 1", FlinkUtil.describe(lastScanEvent.filter()));

Review comment:
       nit: If we use the static imported `assertEquals` before then we don't use `Assert.assertEquals` here ?  Make them to be unified. 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546031497



##########
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 {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(1);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(0);
+    }
+
+    String name = fieldReferenceExpression.getName();
+    Class clazz = valueLiteralExpression.getOutputDataType().getConversionClass();
+    Object value = valueLiteralExpression.getValueAs(clazz).get();

Review comment:
       `ValueLiteralExpression` allows the value to be null, in which case `get` here will throw an exception. How is this avoided? Does the parser reject `col = null` expressions?
   
   @openinx may be able to help here, too.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557558944



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testNotEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.notEqual(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isNotEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isNotEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testNotEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.notNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve(Expressions.$("field1").isNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve(Expressions.$("field1").isNotNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).and(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    And and = (And) actual.get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), and.op());
+    Assert.assertEquals(expected.left().op(), and.left().op());
+    Assert.assertEquals(expected.right().op(), and.right().op());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).or(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Or or = (Or) actual.get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), or.op());
+    Assert.assertEquals(expected.left().op(), or.left().op());

Review comment:
       I think these should use `assertPredicatesMatch` instead of just checking the operation.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558225185



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testNotEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.notEqual(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isNotEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isNotEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testNotEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.notNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve(Expressions.$("field1").isNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve(Expressions.$("field1").isNotNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).and(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    And and = (And) actual.get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), and.op());
+    Assert.assertEquals(expected.left().op(), and.left().op());
+    Assert.assertEquals(expected.right().op(), and.right().op());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).or(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Or or = (Or) actual.get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), or.op());
+    Assert.assertEquals(expected.left().op(), or.left().op());
+    Assert.assertEquals(expected.right().op(), or.right().op());
+  }
+
+  @Test
+  public void testNot() {
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Not not = (Not) actual.get();
+    Not expected = (Not) org.apache.iceberg.expressions.Expressions.not(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1));
+
+    Assert.assertEquals(expected.op(), not.op());
+    assertPredicatesMatch((UnboundPredicate<?>) expected.child(), (UnboundPredicate<?>) not.child());
+  }
+
+  @Test
+  public void testLike() {
+    UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc");
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%")));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  private void matchLiteral(String fieldName, Object flinkLiteral, Object icebergLiteral) {
+    Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral)));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+
+    BoundLiteralPredicate predicate =

Review comment:
       yes, `BoundLiteralPredicate` is a parameterized type, but I define the param `icebergLiteral` as Object type used to accept various types of values,if we change `BoundLiteralPredicate` to `BoundLiteralPredicate<?>` , the `predicate.test` method will throw an exception: `Cannot resolve method 'test(java.lang.Object)'`




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546032827



##########
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 {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(1);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(0);
+    }
+
+    String name = fieldReferenceExpression.getName();
+    Class clazz = valueLiteralExpression.getOutputDataType().getConversionClass();
+    Object value = valueLiteralExpression.getValueAs(clazz).get();
+
+    BuiltInFunctionDefinition functionDefinition = (BuiltInFunctionDefinition) call.getFunctionDefinition();
+    if (functionDefinition.equals(BuiltInFunctionDefinitions.EQUALS) &&
+        functionDefinition.equals(BuiltInFunctionDefinitions.NOT_EQUALS)) {
+      Preconditions.checkNotNull(value, "Expression is always false : %s", call);
+      if (NaNUtil.isNaN(value)) {
+        return Optional.of(Expressions.isNaN(name));
+      } else {
+        return Optional.of(function.apply(name, value));
+      }
+    }

Review comment:
       I think it would be cleaner to have a method to extract the column name and literal value as a pair, rather than passing the function to create an expression in here. Handling the expression type in the caller, but then also handling it here doesn't provide very good separation of concerns.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557560849



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));

Review comment:
       Typo: should be "should not contain"




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546893274



##########
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()));

Review comment:
       Okay, sounds fine that Flink doesn't currently support predicate pushdown on nested fields. @openinx, any plans to change this?




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557814302



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",

Review comment:
       If we put it in one line, it will exceed the max limit of checkstyle, I extract the expected filter to a single string.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557563674



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);

Review comment:
       The expected value comes first so that assertion failures show the right labels. If this failed, the actual filter would be shown as the expected filter.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-749123886


   > I am very sorry than some situations are not well considered
   
   No problem, this is why we review!


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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553150684



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink. the BETWEEN will be convert to (GT_EQ AND
+   * LT_EQ), the NOT_BETWEEN will be convert to (LT_EQ OR GT_EQ), the IN will be convert to OR, so we do not add the
+   * convert here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, call);
+
+        case NOT_EQ:

Review comment:
       OK,  then we have to keep the `NOT_EQ` here. 




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-761800697


   > @zhangjun0x01, there are still a few things to fix in the tests, mostly minor. But I also found a major problem, which is that `TestFlinkTableSource` now takes a very long time to run. The problem is that the tests run for each format and for 3 different catalog configurations. That means each test runs 9 times and because it is a test that actually runs SQL it takes a long time. The whole suite takes much longer than needed; on my machine, it takes about 4 minutes.
   > 
   > The filter pushdown tests only need to run for one catalog and one file format because the purpose of those tests is to validate the assumptions of the `FlinkFilter` class with real filters from Flink SQL. The file format and catalog are orthogonal and we don't need to test each one of them. Can you change the parameterization to run with only Avro and a single catalog case?
   
   I refactor it with `HadoopCatalog` and Avro type.


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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552378963



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink. the BETWEEN will be convert to (GT_EQ AND
+   * LT_EQ), the NOT_BETWEEN will be convert to (LT_EQ OR GT_EQ), the IN will be convert to OR, so we do not add the
+   * convert here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, call);
+
+        case NOT_EQ:

Review comment:
       I tested that,flink do not convert `NOT_EQ` to `NOT`




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-748279868


   Thanks for working on this @zhangjun0x01! It looks like a great start to me, and I'd really like to get this working in Flink.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546048747



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

Review comment:
       This class needs an extensive test suite that checks the conversion from expected Flink expressions, not just a test for the source.
   
   The conversion needs to cover at least these cases:
   * Equals with null
   * Not equals with null
   * In with null
   * Not in with null
   * Equals with NaN
   * Not equals with NaN
   * In with NaN
   * Not in with NaN
   * All inequalities with null
   * All inequalities with NaN
   * All expressions with a non-null and non-Nan value (preferably one string and one numeric)
   * Each data type that is supposed by Iceberg/Flink




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556977750



##########
File path: api/src/main/java/org/apache/iceberg/expressions/ExpressionsUtil.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.expressions;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class ExpressionsUtil {
+  private ExpressionsUtil() {
+  }
+
+  public static String describe(org.apache.iceberg.expressions.Expression expr) {

Review comment:
       I don't think that `Expression` needs to be fully-qualified.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557564922



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",
+            lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("explain should contains the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "a"});
+    expectedGT.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "a"});
+    expectedGTE.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("explain should contains the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("explain should contains the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("should have 1 record", 1, resultNotIn.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)",
+        lastScanEvent.filter().toString());
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));

Review comment:
       Needs to check the scan event.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552379722



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink. the BETWEEN will be convert to (GT_EQ AND
+   * LT_EQ), the NOT_BETWEEN will be convert to (LT_EQ OR GT_EQ), the IN will be convert to OR, so we do not add the
+   * convert here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, call);

Review comment:
       > I think there's a bug here. Assume the case: `a != NaN`, the `handleNaN` will return an iceberg expression: `Expressions.isNaN(name)`. That's incorrect ?
   
   It was my negligence, forgot to deal with `!=`




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


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

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r545365534



##########
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 see. Do we want to return `Optional<Expression>` here then? In this case it signals that the returned value could be null, so when we add the converted expression to the list we can decide to not add nulls, so that we don't have to do null check when calling `toString`?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556992286



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * Convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));

Review comment:
       Similar to above, this shouldn't use `orElse(null)`:
   
   ```java
             return onlyChildAs(call, CallExpression.class).flatMap(FlinkFilters::convert).map(Expressions::not);
   ```




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558842494



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * Convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          return onlyChildAs(call, FieldReferenceExpression.class)
+              .map(FieldReferenceExpression::getName)
+              .map(Expressions::isNull);
+
+        case NOT_NULL:
+          return onlyChildAs(call, FieldReferenceExpression.class)
+              .map(FieldReferenceExpression::getName)
+              .map(Expressions::notNull);
+
+        case LT:
+          return convertFieldAndLiteral(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertFieldAndLiteral(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertFieldAndLiteral(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertFieldAndLiteral(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return convertFieldAndLiteral((ref, lit) -> {
+            if (NaNUtil.isNaN(lit)) {
+              return Expressions.isNaN(ref);
+            } else {
+              return Expressions.equal(ref, lit);
+            }
+          }, call);
+
+        case NOT_EQ:
+          return convertFieldAndLiteral((ref, lit) -> {
+            if (NaNUtil.isNaN(lit)) {
+              return Expressions.notNaN(ref);
+            } else {
+              return Expressions.notEqual(ref, lit);
+            }
+          }, call);
+
+        case NOT:
+          return onlyChildAs(call, CallExpression.class).flatMap(FlinkFilters::convert).map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> onlyChildAs(CallExpression call,
+                                                                        Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args.size() != 2) {
+      return Optional.empty();
+    }
+
+    org.apache.flink.table.expressions.Expression left = args.get(0);
+    org.apache.flink.table.expressions.Expression right = args.get(1);
+
+    if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) {
+      String name = ((FieldReferenceExpression) left).getName();
+      return convertLiteral((ValueLiteralExpression) right).flatMap(lit -> {
+        if (lit instanceof String) {
+          String pattern = (String) lit;
+          Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+          // exclude special char of LIKE
+          // '_' is the wildcard of the SQL LIKE
+          if (!pattern.contains("_") && matcher.matches()) {
+            return Optional.of(Expressions.startsWith(name, matcher.group(1)));
+          }
+        }
+
+        return Optional.empty();
+      });
+    }
+
+    return Optional.empty();
+  }
+
+  private static Optional<Expression> convertLogicExpression(BiFunction<Expression, Expression, Expression> function,
+                                                             CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args == null || args.size() != 2) {
+      return Optional.empty();
+    }
+
+    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<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);
+      }
+
+      return o;
+    });
+  }
+
+  private static Optional<Expression> convertFieldAndLiteral(BiFunction<String, Object, Expression> expr,
+                                                             CallExpression call) {
+    return convertFieldAndLiteral(expr, expr, call);
+  }
+
+  private static Optional<Expression> convertFieldAndLiteral(
+      BiFunction<String, Object, Expression> convertLR, BiFunction<String, Object, Expression> convertRL,
+      CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args.size() != 2) {
+      return Optional.empty();
+    }
+
+    org.apache.flink.table.expressions.Expression left = args.get(0);
+    org.apache.flink.table.expressions.Expression right = args.get(1);
+
+    if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) {
+      String name = ((FieldReferenceExpression) left).getName();
+      Optional<Object> lit = convertLiteral((ValueLiteralExpression) right);
+      if (lit.isPresent()) {
+        return Optional.of(convertLR.apply(name, lit.get()));

Review comment:
       `Optional#map` method required a `Function` param, but this method provide a `BiFunction` param,so I use `if` to do the judgment




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553158808



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);

Review comment:
       In this `matchLiteral` testing method,  we will provide both flink's java data type and iceberg's java data type ( for asserting).   I think it's not a good way to test those literals because if people don't quite understand what's the specific java data type that iceberg type is mapping,   then he may provide the wrong iceberg's java type to assert.  The `BINARY` type is an example. 
   




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r544852609



##########
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:
       If it is an unsupported expression, there is no need to do filter push down, I think we should not throw a exception




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546487254



##########
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 {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(1);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(0);
+    }
+
+    String name = fieldReferenceExpression.getName();
+    Class clazz = valueLiteralExpression.getOutputDataType().getConversionClass();
+    Object value = valueLiteralExpression.getValueAs(clazz).get();

Review comment:
       I tested it ,if the sql is `select * from mytable where data = null` ,it do not supports filter push down in flink,and we do not can get any data.
   if the sql is `select * from mytable where data is null`, it is normal ,and It will enter the `IS_NULL` branch of switch 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546027209



##########
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());

Review comment:
       My IDE warns that this is a suspicious call because it gets a `FunctionDefinition` from a map keyed by `BuiltInFunctionDefinition`. To fix it, I think the map should be `Map<FunctionDefinition, Operation>`.

##########
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();
+

Review comment:
       Nit: extra newline.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558841800



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",
+            lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("explain should contains the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "a"});
+    expectedGT.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "a"});
+    expectedGTE.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("explain should contains the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("explain should contains the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("should have 1 record", 1, resultNotIn.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)",
+        lastScanEvent.filter().toString());
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "a"});
+    expected.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultNotNull.toArray(), expected.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNull.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNot() {

Review comment:
       I update it




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


[GitHub] [iceberg] rdblue edited a comment on pull request #1893: Flink: Support filter pushdown in IcebergTableSource

Posted by GitBox <gi...@apache.org>.
rdblue edited a comment on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-761613165


   @zhangjun0x01, there are still a few things to fix in the tests, mostly minor. But I also found a major problem, which is that `TestFlinkTableSource` now takes a very long time to run. The problem is that the tests run for each format and for 3 different catalog configurations. That means each test runs 9 times and because it is a test that actually runs SQL it takes a long time. The whole suite takes much longer than needed; on my machine, it took 20 minutes with code coverage turned on!
   
   The filter pushdown tests only need to run for one catalog and one file format because the purpose of those tests is to validate the assumptions of the `FlinkFilter` class with real filters from Flink SQL. The file format and catalog are orthogonal and we don't need to test each one of them. Can you change the parameterization to run with only Avro and a single catalog case?


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


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

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r545365534



##########
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 see. Do we want to return `Optional<Expression>` here then? In this case it signals that the returned value could be null, so when we add the converted expression to the list we can decide to not add nulls. 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553134655



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)

Review comment:
       This key-value pairs could be removed , right ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);

Review comment:
       If the literal type of flink is BINARY or VARBINARY,   then its java data type is  `byte[]`,  while in iceberg we will use `ByteBuffer` to do the literal comparison.  So I think we need to convert it to ByteBuffer by `ByteBuffer.wrap((byte[])o)` .  Otherwise the iceberg literal comparing will throw a cast failure . 
   
   Pls see the java data type for iceberg type  here: https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/types/Type.java#L30
   
   By the way,  I think you unit tests here https://github.com/apache/iceberg/pull/1893/files#diff-5d18d1ff127d1dc70a9a15bbe941f2b6f9d28b3015924f601ac1f722914099dbR81 is incorrect.  

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {

Review comment:
       For the flink's `Binary` data type,  its default java type is  `byte[]`,  while the iceberg's `BinaryLiteral`  will use `ByteBuffer`.    So we will need to convert it to ByteBuffer ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);
+      }
+
+      return o;
+    });
+  }
+
+  private static boolean literalOnRight(List<ResolvedExpression> args) {
+    return args.get(0) instanceof FieldReferenceExpression && args.get(1) instanceof ValueLiteralExpression;
+  }
+
+  private static Tuple2<String, Object> convertBinaryExpress(CallExpression call) {

Review comment:
       Nit:  It's better to rename it as `parseFieldAndLiteral` ?   The method is not actually convert expressions from flink expr to iceberg expr ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {

Review comment:
       Pls see the iceberg expression literal types here : https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/types/Type.java#L30




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r544826527



##########
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:
       iceberg support the following Expressions:
   https://iceberg.apache.org/api/#expressions 
   For some expressions supported by flink but not supported by iceberg, I did not convert them, because they cannot be used for iceberg table scan
   




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552378770



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkUtil.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.BoundPredicate;
+import org.apache.iceberg.expressions.ExpressionVisitors;
+import org.apache.iceberg.expressions.Literal;
+import org.apache.iceberg.expressions.UnboundPredicate;
+
+public class FlinkUtil {

Review comment:
       I copyed the  code from the spark module, in spark, there are production files call this mothed.
   
   In Flink, this class does only have methods related to filters. I think some methods related to flink util can be added to this class in the future. So I do not add the class in test packages,what do you think?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546028797



##########
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()));

Review comment:
       How does `FieldReferenceExpression.getName()` reference nested fields?




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547982574



##########
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: {

Review comment:
       nit:  the `{` and `}` could be removed .

##########
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: {

Review comment:
       ditto




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551206609



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +95,328 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
     Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, mixedResult.size());
+    assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    assertTrue("explain should contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id = 1", FlinkUtil.describe(lastScanEvent.filter()));
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    assertFalse("explain should not contains FilterPushDown", explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    assertTrue("explain should contains FilterPushDown", explainLeft.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    assertEquals("should have 1 record", 1, resultLeft.size());
+    assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id = 1", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    assertTrue("explain should contains FilterPushDown", explainNE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    assertEquals("should have 2 record", 2, resultNE.size());
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id != 1", FlinkUtil.describe(lastScanEvent.filter()));
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    assertTrue("explain should contains FilterPushDown", explainAnd.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    assertEquals("should have 1 record", 1, resultAnd.size());
+    assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "(id = 1 AND data = 'a')",
+        FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    assertTrue("explain should contains FilterPushDown", explainOr.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultOr = sql(sqlOr);
+    assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "(id = 1 OR data = 'b')",
+        FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    assertTrue("explain should contains FilterPushDown", explainGT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    assertEquals("should have 2 record", 2, resultGT.size());
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id > 1", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    assertTrue("explain should contains FilterPushDown", explainGTE.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultGTE = sql(sqlGTE);
+    assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id >= 2", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    assertTrue("explain should contains FilterPushDown", explainLT.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultLT = sql(sqlLT);
+    assertEquals("should have 1 record", 1, resultLT.size());
+    assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id < 2", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    assertTrue("explain should contains FilterPushDown", explainLTE.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultLTE = sql(sqlLTE);
+    assertEquals("should have 1 record", 1, resultLTE.size());
+    assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id <= 1", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    assertTrue("explain should contains FilterPushDown", explainIN.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultIN = sql(sqlIN);
+    assertEquals("should have 2 records", 2, resultIN.size());
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "(id = 1 OR id = 2)",
+        FlinkUtil.describe(lastScanEvent.filter()));
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    assertFalse("explain should not contains FilterPushDown", explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    assertTrue("explain should contains FilterPushDown", explainNotIn.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    assertEquals("should have 1 record", 1, resultNotIn.size());
+    assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "(id != 3 AND id != 2)",
+        FlinkUtil.describe(lastScanEvent.filter()));
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    assertFalse("explain should not contains FilterPushDown", explainNotInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    assertTrue("explain should contains FilterPushDown", explainNotNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    assertEquals("should have 2 record", 2, resultNotNull.size());
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "a"});
+    expected.add(new Object[] {2, "b"});
+    assertArrayEquals("Should produce the expected record", resultNotNull.toArray(), expected.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "data IS NOT NULL",
+        FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    assertTrue("explain should contains FilterPushDown", explainNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultNull = sql(sqlNull);
+    assertEquals("should have 1 record", 1, resultNull.size());
+    assertArrayEquals("Should produce the expected record", resultNull.get(0), new Object[] {3, null});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    assertEquals("Should push down expected filter", "data IS NULL", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT id = 1 ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    assertTrue("explain should contains FilterPushDown", explainNot.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultNot = sql(sqlNot);
+    assertEquals("should have 2 record", 2, resultNot.size());
+    List<Object[]> expectedNot = Lists.newArrayList();
+    expectedNot.add(new Object[] {2, "b"});
+    expectedNot.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected record", resultNot.toArray(), expectedNot.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id != 1", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    assertTrue("explain should contains FilterPushDown", explainBetween.contains(expectedFilterPushDownExplain));

Review comment:
       How about changing this to check whether the `explainBetween` contains a more detailed string `FilterPushDown,the filters :ref(name="id") >= 1,ref(name="id") <= 2]]], fields=[id, data])` ?   `explainBetween.contains("FilterPushDown")` is not so accurate for me. 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557551619



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testNotEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.notEqual(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isNotEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isNotEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testNotEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.notNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve(Expressions.$("field1").isNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve(Expressions.$("field1").isNotNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).and(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    And and = (And) actual.get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), and.op());
+    Assert.assertEquals(expected.left().op(), and.left().op());
+    Assert.assertEquals(expected.right().op(), and.right().op());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).or(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Or or = (Or) actual.get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), or.op());
+    Assert.assertEquals(expected.left().op(), or.left().op());
+    Assert.assertEquals(expected.right().op(), or.right().op());
+  }
+
+  @Test
+  public void testNot() {
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Not not = (Not) actual.get();
+    Not expected = (Not) org.apache.iceberg.expressions.Expressions.not(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1));
+
+    Assert.assertEquals(expected.op(), not.op());
+    assertPredicatesMatch((UnboundPredicate<?>) expected.child(), (UnboundPredicate<?>) not.child());
+  }
+
+  @Test
+  public void testLike() {
+    UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc");
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%")));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  private void matchLiteral(String fieldName, Object flinkLiteral, Object icebergLiteral) {
+    Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral)));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());

Review comment:
       Assertions should have context so that when one fails it is clear what went wrong. In this case, the assertion is testing that converting succeeded. So this should be `Assert.assertTrue("Conversion should succeed", actual.isPresent())`




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551984286



##########
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:
       nit:  it's more simple to rewrite this as: 
   
   ```java
    explain += String.format(", FilterPushDown,the filters :%s", Joiner.on(",").join(filters));
   ```




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-752309028


   @zhangjun0x01  Would you mind to resolve the conflicts ?    I will take another look at this PR if you resolve it. Thanks.


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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547966955



##########
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 {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(1);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(0);
+    }
+
+    String name = fieldReferenceExpression.getName();
+    Class clazz = valueLiteralExpression.getOutputDataType().getConversionClass();
+    Object value = valueLiteralExpression.getValueAs(clazz).get();

Review comment:
       I agreed with @rdblue  that handling null in this function rather than assuming flink won't push down the `null`. 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557016887



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +95,436 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
-    Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
+    assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, mixedResult.size());
+    assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String explainExpected = "FilterPushDown,the filters :ref(name=\"id\") == 1]]], fields=[id, data])";
+    assertTrue("explain should contains FilterPushDown", explain.contains(explainExpected));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    assertEquals("Should push down expected filter", "id = 1", ExpressionsUtil.describe(lastScanEvent.filter()));
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    assertFalse("explain should not contains FilterPushDown", explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String explainExpected = "FilterPushDown,the filters :ref(name=\"id\") == 1]]], fields=[id, data])";
+    assertTrue("explain should contains FilterPushDown", explainLeft.contains(explainExpected));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    assertEquals("should have 1 record", 1, resultLeft.size());
+    assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    assertEquals("Should push down expected filter", "id = 1", ExpressionsUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String explainExpected = "FilterPushDown,the filters :ref(name=\"id\") != 1]]], fields=[id, data])";
+    assertTrue("explain should contains FilterPushDown", explainNE.contains(explainExpected));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    assertEquals("Should push down expected filter", "id != 1", ExpressionsUtil.describe(lastScanEvent.filter()));
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String explainExpected =
+        "FilterPushDown,the filters :ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"]]], fields=[id, data])";
+    assertTrue("explain should contains FilterPushDown", explainAnd.contains(explainExpected));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    assertEquals("should have 1 record", 1, resultAnd.size());
+    assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    assertEquals("Should push down expected filter", "(id = 1 AND data = 'a')",
+        ExpressionsUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String explainExpected =
+        "FilterPushDown,the filters :(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")]]], fields=[id, data])";
+    assertTrue("explain should contains FilterPushDown", explainOr.contains(explainExpected));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    assertEquals("Should push down expected filter", "(id = 1 OR data = 'b')",
+        ExpressionsUtil.describe(lastScanEvent.filter()));

Review comment:
       It looks like `describe` was moved just for tests. I don't think that was needed. If you need to produce a string from an expression, why not just use `toString`? That's what other assertions use, like the one for `explainExpected` above.
   
   Can you try reverting the `describe` change?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557553866



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testNotEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.notEqual(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isNotEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isNotEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testNotEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.notNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve(Expressions.$("field1").isNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve(Expressions.$("field1").isNotNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).and(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    And and = (And) actual.get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), and.op());
+    Assert.assertEquals(expected.left().op(), and.left().op());
+    Assert.assertEquals(expected.right().op(), and.right().op());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).or(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Or or = (Or) actual.get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), or.op());
+    Assert.assertEquals(expected.left().op(), or.left().op());
+    Assert.assertEquals(expected.right().op(), or.right().op());
+  }
+
+  @Test
+  public void testNot() {
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Not not = (Not) actual.get();
+    Not expected = (Not) org.apache.iceberg.expressions.Expressions.not(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1));
+
+    Assert.assertEquals(expected.op(), not.op());
+    assertPredicatesMatch((UnboundPredicate<?>) expected.child(), (UnboundPredicate<?>) not.child());
+  }
+
+  @Test
+  public void testLike() {
+    UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc");
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%")));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  private void matchLiteral(String fieldName, Object flinkLiteral, Object icebergLiteral) {
+    Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral)));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+
+    BoundLiteralPredicate predicate =
+        (BoundLiteralPredicate<?>) ((UnboundPredicate<?>) actual.get())
+            .bind(FlinkSchemaUtil.convert(TABLE_SCHEMA).asStruct(), false);
+    Assert.assertTrue(predicate.test(icebergLiteral));
+  }
+
+  private static Expression resolve(Expression originalExpression) {
+    return originalExpression.accept(new ApiExpressionDefaultVisitor<Expression>() {
+      @Override
+      public Expression visit(UnresolvedReferenceExpression unresolvedReference) {
+        String name = unresolvedReference.getName();
+        Optional<TableColumn> field = TABLE_SCHEMA.getTableColumn(name);
+        if (field.isPresent()) {
+          int index = TABLE_SCHEMA.getTableColumns().indexOf(field.get());
+          return new FieldReferenceExpression(name, field.get().getType(), 0, index);
+        } else {
+          return null;
+        }
+      }
+
+      @Override
+      public Expression visit(UnresolvedCallExpression unresolvedCall) {
+        List<ResolvedExpression> children =
+            unresolvedCall.getChildren().stream().map(e -> (ResolvedExpression) e.accept(this))
+                .collect(Collectors.toList());
+        return new CallExpression(unresolvedCall.getFunctionDefinition(), children, DataTypes.STRING());
+      }
+
+      @Override
+      public Expression visit(ValueLiteralExpression valueLiteral) {
+        return valueLiteral;
+      }
+
+      @Override
+      protected Expression defaultMethod(Expression expression) {
+        throw new UnsupportedOperationException(String.format("unsupported expression: %s", expression));
+      }
+    });
+  }
+
+  private void assertPredicatesMatch(UnboundPredicate<?> expected, UnboundPredicate<?> actual) {
+    Assert.assertEquals(expected.op(), actual.op());
+    Assert.assertEquals(expected.literal(), actual.literal());
+    Assert.assertEquals(expected.ref().name(), actual.ref().name());

Review comment:
       These assertions also need context strings to identify which part of the predicate is tested, like "Predicate operation should match"




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r548780579



##########
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:
       I add the test case [literal on right](https://github.com/zhangjun0x01/iceberg/blob/filterPushDown/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java#L133),[ literal on left](https://github.com/zhangjun0x01/iceberg/blob/filterPushDown/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java#L152) ,[2 literals](https://github.com/zhangjun0x01/iceberg/blob/filterPushDown/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java#L416)




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553710377



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +95,396 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
     Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, mixedResult.size());
+    assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    assertTrue("explain should contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});

Review comment:
       I has tested that , if we use `assertEquals` for two lists,it will be failed, so I convert the list to array.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552389034



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink. the BETWEEN will be convert to (GT_EQ AND
+   * LT_EQ), the NOT_BETWEEN will be convert to (LT_EQ OR GT_EQ), the IN will be convert to OR, so we do not add the
+   * convert here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, call);

Review comment:
       > Pls add an unit test for this if possible.
   
   we can construct `NaN` type by `float('NaN')` in spark and hive sql,
   but I lookup the flink [doc](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/types.html),It does not seem to support `Infinity` an `NaN` now




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552378770



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkUtil.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.BoundPredicate;
+import org.apache.iceberg.expressions.ExpressionVisitors;
+import org.apache.iceberg.expressions.Literal;
+import org.apache.iceberg.expressions.UnboundPredicate;
+
+public class FlinkUtil {

Review comment:
       I copyed the  code from the spark module, in spark, there are production files call this method.
   
   In Flink, this class does only have methods related to filters. I think some methods related to flink util can be added to this class in the future. So I do not add the class in test packages,what do you think?




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552003350



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, new byte[] {'a', 'b'});
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+
+    matchLiteral("field13", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.equal("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isEqual(Expressions.lit(1)))).get();

Review comment:
       Nit: better to check `ifPresent` before `get` the value from `Optional`. 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557563106



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",

Review comment:
       Nit: this newline isn't needed.




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551201072



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  public static Optional<Expression> convert(org.apache.flink.table.expressions.Expression flinkExpression) {
+    if (flinkExpression == null || !(flinkExpression instanceof CallExpression)) {

Review comment:
       Here we could remove the `flinkExpression == null` and just keep the 
   ```java
       if (!(flinkExpression instanceof CallExpression)) {
         return Optional.empty();
       }
   ```
   
   Because `null` will always meet the `!(flinkExpression instanceof CallExpression)` condition.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556978618



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to

Review comment:
       Nits: missing a space in the list and the next sentence isn't capitalized.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556978350



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.

Review comment:
       Nit: "convert" should be capitalized because it begins a sentence.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551696238



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +95,328 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
     Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, mixedResult.size());
+    assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    assertTrue("explain should contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id = 1", FlinkUtil.describe(lastScanEvent.filter()));
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    assertFalse("explain should not contains FilterPushDown", explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    assertTrue("explain should contains FilterPushDown", explainLeft.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    assertEquals("should have 1 record", 1, resultLeft.size());
+    assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id = 1", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    assertTrue("explain should contains FilterPushDown", explainNE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    assertEquals("should have 2 record", 2, resultNE.size());
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id != 1", FlinkUtil.describe(lastScanEvent.filter()));
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    assertTrue("explain should contains FilterPushDown", explainAnd.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    assertEquals("should have 1 record", 1, resultAnd.size());
+    assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "(id = 1 AND data = 'a')",
+        FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    assertTrue("explain should contains FilterPushDown", explainOr.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultOr = sql(sqlOr);
+    assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "(id = 1 OR data = 'b')",
+        FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    assertTrue("explain should contains FilterPushDown", explainGT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    assertEquals("should have 2 record", 2, resultGT.size());
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id > 1", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    assertTrue("explain should contains FilterPushDown", explainGTE.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultGTE = sql(sqlGTE);
+    assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id >= 2", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    assertTrue("explain should contains FilterPushDown", explainLT.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultLT = sql(sqlLT);
+    assertEquals("should have 1 record", 1, resultLT.size());
+    assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id < 2", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    assertTrue("explain should contains FilterPushDown", explainLTE.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultLTE = sql(sqlLTE);
+    assertEquals("should have 1 record", 1, resultLTE.size());
+    assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id <= 1", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    assertTrue("explain should contains FilterPushDown", explainIN.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultIN = sql(sqlIN);
+    assertEquals("should have 2 records", 2, resultIN.size());
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "(id = 1 OR id = 2)",
+        FlinkUtil.describe(lastScanEvent.filter()));
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    assertFalse("explain should not contains FilterPushDown", explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    assertTrue("explain should contains FilterPushDown", explainNotIn.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    assertEquals("should have 1 record", 1, resultNotIn.size());
+    assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "(id != 3 AND id != 2)",
+        FlinkUtil.describe(lastScanEvent.filter()));
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    assertFalse("explain should not contains FilterPushDown", explainNotInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    assertTrue("explain should contains FilterPushDown", explainNotNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    assertEquals("should have 2 record", 2, resultNotNull.size());
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "a"});
+    expected.add(new Object[] {2, "b"});
+    assertArrayEquals("Should produce the expected record", resultNotNull.toArray(), expected.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "data IS NOT NULL",
+        FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    assertTrue("explain should contains FilterPushDown", explainNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultNull = sql(sqlNull);
+    assertEquals("should have 1 record", 1, resultNull.size());
+    assertArrayEquals("Should produce the expected record", resultNull.get(0), new Object[] {3, null});
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    assertEquals("Should push down expected filter", "data IS NULL", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT id = 1 ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    assertTrue("explain should contains FilterPushDown", explainNot.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultNot = sql(sqlNot);
+    assertEquals("should have 2 record", 2, resultNot.size());
+    List<Object[]> expectedNot = Lists.newArrayList();
+    expectedNot.add(new Object[] {2, "b"});
+    expectedNot.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected record", resultNot.toArray(), expectedNot.toArray());
+
+    assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter", "id != 1", FlinkUtil.describe(lastScanEvent.filter()));
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    assertTrue("explain should contains FilterPushDown", explainBetween.contains(expectedFilterPushDownExplain));

Review comment:
       I add the more detail string in `IN`,`BETWEEN`,`NO_BETWEEN` test case




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553711485



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);

Review comment:
       I updated it




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557815890



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",

Review comment:
       Please wrap the line at the start of the expected string instead.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557542475



##########
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);
+  }
+
+  @Override
+  public TableSource<RowData> applyPredicate(List<Expression> predicates) {
+    List<org.apache.iceberg.expressions.Expression> expressions = Lists.newArrayList();
+    for (Expression predicate : predicates) {
+      FlinkFilters.convert(predicate).ifPresent(expressions::add);
+    }
+
+    return new IcebergTableSource(loader, schema, properties, projectedFields, isLimitPushDown, limit, expressions);

Review comment:
       @openinx, is this guaranteed to only be called on a source that has not had predicates pushed? This ignores the existing predicates in this source. Maybe we should add a precondition to check that assumption.
   
   I think it is safe either way because this doesn't remove any predicates from the list. From reading the Javadoc, I think that will result in all predicates running in Flink also.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557569464



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",
+            lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("explain should contains the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "a"});
+    expectedGT.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "a"});
+    expectedGTE.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("explain should contains the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("explain should contains the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("should have 1 record", 1, resultNotIn.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)",
+        lastScanEvent.filter().toString());
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "a"});
+    expected.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultNotNull.toArray(), expected.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNull.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT id = 1 ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("should have 2 record", 2, resultNot.size());
+
+    List<Object[]> expectedNot = Lists.newArrayList();
+    expectedNot.add(new Object[] {2, "b"});
+    expectedNot.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNot.toArray(), expectedNot.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "a"});
+    expectedBetween.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultBetween.toArray(), expectedBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)",
+        lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("explain should contains the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'a%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"a\"\"";
+    Assert
+        .assertTrue("the like sql explain should contains the push down filter", explainLike.contains(expectedFilter));
+
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("the like result should produce the expected record", resultLike.get(0),
+        new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // not push down
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%a%' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDown2Literal() {
+    String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME);
+    String explain2Literal = getTableEnv().explainSql(sql2Literal);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explain2Literal.contains(expectedFilterPushDownExplain));
+  }

Review comment:
       @zhangjun0x01, I think this suite also needs to test the behavior of a few more cases to ensure that the assumptions about Flink's parser hold. Even if these result in parser errors, those are good tests to have to validate that if Flink ever does support parsing the expressions, Iceberg will do the right thing.
   
   * `double_col = NaN`
   * `double_col <> NaN`
   * All inequalities with NaN, like `double_col < NaN`
   * All inequalities with null, like `data < null`




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553724321



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);

Review comment:
       I  checkout the code ,I found that iceberg already do the convertion, ([here](https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/expressions/Literals.java#L77)) ,so I did not add the conversion here.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556983495



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));

Review comment:
       I don't think that this should pass `null` to `toReference`. It works because `toReference` does an `instanceof` check, but it isn't obvious that `null` is expected there.
   
   I think it would be better to use `flatMap` to run `toReference` if the `FieldReferenceExpression` is defined.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557555343



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(

Review comment:
       Could you update this to use the Iceberg `Pair` class instead?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556985138



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,

Review comment:
       I don't think that this method name is enough to see what's going on from reading it. How about a name like `childAs` or `onlyChildAs`? That makes it clear that the call's child will be returned and that there should be only one.




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551899511



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink. the BETWEEN will be convert to (GT_EQ AND
+   * LT_EQ), the NOT_BETWEEN will be convert to (LT_EQ OR GT_EQ), the IN will be convert to OR, so we do not add the
+   * convert here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, call);

Review comment:
       I think there's a bug here.   Assume the case:  `a != NaN`,   the `handleNaN` will return an iceberg expression:  `Expressions.isNaN(name)`.  That's incorrect ? 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558995913



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +92,584 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
-    Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
+    Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected records", new Object[] {1, "iceberg", 10.0}, result.get(0));
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    Assert.assertEquals("Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("Should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
-    expectedList.add(new Object[] {1, "a"});
-    expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {1, "iceberg", 10.0});
+    expectedList.add(new Object[] {2, "b", 20.0});
+    expectedList.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected records", expectedList.toArray(), resultExceed.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, mixedResult.size());
+    Assert.assertArrayEquals("Should produce the expected records",
+        new Object[] {1, "iceberg", 10.0}, mixedResult.get(0));
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownEqualNull() {
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlEqualNull);
+    Assert.assertEquals("Should have 0 record", 0, result.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("Should have 1 record", 1, resultLeft.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLeft.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("Should have 2 records", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b", 20.0});
+    expectedNE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedNE.toArray(), resultNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqualNull() {
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+
+    List<Object[]> resultNE = sql(sqlNotEqualNull);
+    Assert.assertEquals("Should have 0 records", 0, resultNE.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"iceberg\"";
+    Assert.assertTrue("Explain should contain the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("Should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultAnd.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("Explain should contain the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("Should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "iceberg", 10.0});
+    expectedOR.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedOR.toArray(), resultOr.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    expectedGT.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanNull() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 records", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "iceberg", 10.0});
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    expectedGTE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualNull() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "iceberg", 10.0});
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanNull() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualNull() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("Explain should contain the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("Should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "iceberg", 10.0});
+    expectedIN.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedIN.toArray(), resultIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownInNull() {
+    String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlInNull);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("Should have 1 record", 1, resultNotIn.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultNotIn.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expectedScan = "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedScan, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotInNull() {
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultGT = sql(sqlNotInNull);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("Should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "iceberg", 10.0});
+    expected.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expected.toArray(), resultNotNull.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("Should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNull.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("Should have 1 record", 1, resultNot.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNot.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("Should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "iceberg", 10.0});
+    expectedBetween.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedBetween.toArray(), resultBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between Should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "iceberg", 10.0});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\"";
+    Assert.assertTrue("the like sql Explain should contain the push down filter", explainLike.contains(expectedFilter));
+
+    sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' ";
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("The like result should produce the expected record",
+        new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterNotPushDownLike() {

Review comment:
       I think this is missing a case for `i%g`. It has a case for `%ice%g`, but I think it should test both.




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


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

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r545367622



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.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 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> 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 Expressions.in(field.getName(), inputValues);
+
+        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();
+    Object value = valueLiteralExpression.getValueAs(clazz).get();
+
+    BuiltInFunctionDefinition functionDefinition = (BuiltInFunctionDefinition) call.getFunctionDefinition();
+    if (functionDefinition.equals(BuiltInFunctionDefinitions.EQUALS)) {

Review comment:
       I think we may want to rewrite `NOT_EQUALS` to `notNaN` as well as `notEquals` in Iceberg also doesn't accept NaN as literal; I think SparkFilters doesn't do that because there's no `NotEqualTo` filter in Spark. 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557551896



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testNotEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.notEqual(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isNotEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isNotEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testNotEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.notNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve(Expressions.$("field1").isNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve(Expressions.$("field1").isNotNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).and(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    And and = (And) actual.get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), and.op());
+    Assert.assertEquals(expected.left().op(), and.left().op());
+    Assert.assertEquals(expected.right().op(), and.right().op());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).or(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Or or = (Or) actual.get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), or.op());
+    Assert.assertEquals(expected.left().op(), or.left().op());
+    Assert.assertEquals(expected.right().op(), or.right().op());
+  }
+
+  @Test
+  public void testNot() {
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Not not = (Not) actual.get();
+    Not expected = (Not) org.apache.iceberg.expressions.Expressions.not(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1));
+
+    Assert.assertEquals(expected.op(), not.op());
+    assertPredicatesMatch((UnboundPredicate<?>) expected.child(), (UnboundPredicate<?>) not.child());
+  }
+
+  @Test
+  public void testLike() {
+    UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc");
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%")));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  private void matchLiteral(String fieldName, Object flinkLiteral, Object icebergLiteral) {
+    Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral)));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+
+    BoundLiteralPredicate predicate =

Review comment:
       `BoundLiteralPredicate` is a parameterized type, so it should have parameters here.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557815753



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testNotEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.notEqual(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isNotEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isNotEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testNotEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.notNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve(Expressions.$("field1").isNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve(Expressions.$("field1").isNotNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).and(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    And and = (And) actual.get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), and.op());
+    Assert.assertEquals(expected.left().op(), and.left().op());
+    Assert.assertEquals(expected.right().op(), and.right().op());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).or(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Or or = (Or) actual.get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), or.op());
+    Assert.assertEquals(expected.left().op(), or.left().op());
+    Assert.assertEquals(expected.right().op(), or.right().op());
+  }
+
+  @Test
+  public void testNot() {
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Not not = (Not) actual.get();
+    Not expected = (Not) org.apache.iceberg.expressions.Expressions.not(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1));
+
+    Assert.assertEquals(expected.op(), not.op());
+    assertPredicatesMatch((UnboundPredicate<?>) expected.child(), (UnboundPredicate<?>) not.child());
+  }
+
+  @Test
+  public void testLike() {

Review comment:
       Also, could you test the wildcard behavior with `_`?




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558228728



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -73,15 +73,15 @@ protected TableEnvironment getTableEnv() {
   }
 
   protected static TableResult exec(TableEnvironment env, String query, Object... args) {
-    return env.executeSql(String.format(query, args));
+    return env.executeSql(args.length > 0 ? String.format(query, args) : query);
   }
 
   protected TableResult exec(String query, Object... args) {
     return exec(getTableEnv(), query, args);
   }
 
   protected List<Object[]> sql(String query, Object... args) {
-    TableResult tableResult = exec(String.format(query, args));
+    TableResult tableResult = exec(query, args);

Review comment:
       I remove the ` return env.executeSql(args.length > 0 ? String.format(query, args) : query);` from `exec` method, but if we retain the `String.format` in `sql(String query, Object... args)` method ,the sql will still be formatted twice,even though we use the `%%` , the sql also is formated failed




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546042929



##########
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));

Review comment:
       There are several calls to `getResolvedChildren().get(0)`. I think that should be converted to a method that validates there is only one child and also validates the type:
   
   ```java
     private <T extends ResolvedExpression> Optional<T> getOnlyChild(CallExpression call, Class<T> expectedChildClass) {
       List<ResolvedExpression> children = call.getResolvedChildren();
       if (children.size() != 1) {
         return Optional.empty();
       }
   
       ResolvedExpression child = children.get(0);
       if (!expectedChildClass.isInstance(child)) {
         return Optional.empty();
       }
   
       return Optional.of(expectedChildClass.cast(child));
     }
   ```




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557017294



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;

Review comment:
       Can you please remove the static imports?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557556491



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());

Review comment:
       It would be better to check the cast instead of failing tests with `ClassCastException` if the wrong class is returned. Could you update `assertPredicatesMatch` to accept `actual` as an `Expression` and add an `instanceof` assertion in that method?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557816528



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",
+            lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("explain should contains the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "a"});
+    expectedGT.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "a"});
+    expectedGTE.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("explain should contains the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("explain should contains the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("should have 1 record", 1, resultNotIn.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)",
+        lastScanEvent.filter().toString());
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "a"});
+    expected.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultNotNull.toArray(), expected.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNull.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT id = 1 ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("should have 2 record", 2, resultNot.size());
+
+    List<Object[]> expectedNot = Lists.newArrayList();
+    expectedNot.add(new Object[] {2, "b"});
+    expectedNot.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNot.toArray(), expectedNot.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "a"});
+    expectedBetween.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultBetween.toArray(), expectedBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)",
+        lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("explain should contains the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'a%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"a\"\"";
+    Assert
+        .assertTrue("the like sql explain should contains the push down filter", explainLike.contains(expectedFilter));
+
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("the like result should produce the expected record", resultLike.get(0),
+        new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // not push down
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%a%' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDown2Literal() {
+    String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME);
+    String explain2Literal = getTableEnv().explainSql(sql2Literal);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explain2Literal.contains(expectedFilterPushDownExplain));
+  }

Review comment:
       Yes, if those are not supported, then we should have tests that assert a parse error. We want the tests to start failing if we update to a version that does support `NaN` in the parser and it is used in an expression.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546029936



##########
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 {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(1);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(0);
+    }
+
+    String name = fieldReferenceExpression.getName();
+    Class clazz = valueLiteralExpression.getOutputDataType().getConversionClass();

Review comment:
       `Class` is parameterized, so this should be `Class<?>`




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552388800



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, new byte[] {'a', 'b'});
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+
+    matchLiteral("field13", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.equal("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isEqual(Expressions.lit(1)))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isEqual(Expressions.$("field1")))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);

Review comment:
       I add the test case 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547983724



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

Review comment:
       The flink's `LIKE`, `BETWEEN`, `NOT_BETWEEN` could also be pushed down, pls see https://github.com/generic-datalake/iceberg-poc/pull/2/files#diff-86160616589acf1dd526b10b73418a46fe60f9e5e5ab6946a4ea3c8f019542f5R78-R80.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-748924326


   > Thanks for working on this @zhangjun0x01! It looks like a great start to me, and I'd really like to get this working in Flink.
   
   @rdblue , thank you very much for your review,I am very sorry than some situations are not well considered,I would be careful next time, and I will update the PR later 


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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r548778689



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

Review comment:
       I tested that ,I found `BETWEEN`, `NOT_BETWEEN`,`IN` expression will be auto convert. 
   the `BETWEEN` will be convert to `(GT_EQ AND LT_EQ)`,
   the `NOT_BETWEEN` will be convert to `(LT_EQ OR GT_EQ)`,
   the `IN` will be convert to `OR`,
   
   in `IcebergTableSource#applyPredicate` method ,We won't get `BETWEEN`,`NOT_BETWEEN`,`IN` expressions,so I do not add `BETWEEN`,`NOT_BETWEEN` in `FlinkFilters`, and do not add test case in `TestFlinkFilters`.but add test  case in `TestFlinkTableSource` to valiate the convert.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556987036



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);
+      }
+
+      return o;
+    });
+  }
+
+  private static boolean literalOnRight(List<ResolvedExpression> args) {
+    return args.get(0) instanceof FieldReferenceExpression && args.get(1) instanceof ValueLiteralExpression;
+  }
+
+  private static Tuple2<String, Object> parseFieldAndLiteral(CallExpression call) {

Review comment:
       I think this should use Iceberg's `Pair` instead of `Tuple2`.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r544829918



##########
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()`?
   
   `flinkExpression.getFunctionDefinition()`  return a implement class of `FunctionDefinition`,which cannot be used directly in switch,so we add a mapping,similar to [SparkFilters](https://github.com/apache/iceberg/blob/master/spark3/src/main/java/org/apache/iceberg/spark/SparkFilters.java#L70) 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553165643



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +95,396 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
     Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, mixedResult.size());
+    assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    assertTrue("explain should contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});

Review comment:
       Nit:  we could just use the `assertEquals` for two lists, don't have to convert it to array and then use `assertArrayEquals` ? 




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-761550953


   > I'd like to get this into the 0.11.0 release, if possible. Thanks for working on this, @zhangjun0x01! It will be great to have this feature done.
   
   @rdblue thanks very much for your review,I updated it.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557543036



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -73,15 +73,15 @@ protected TableEnvironment getTableEnv() {
   }
 
   protected static TableResult exec(TableEnvironment env, String query, Object... args) {
-    return env.executeSql(String.format(query, args));
+    return env.executeSql(args.length > 0 ? String.format(query, args) : query);

Review comment:
       Why was this change needed?




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547976691



##########
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);

Review comment:
       Q:  the `call.getResolvedChildren().get(0)` MUST be a `FieldReferenceExpression` ?   I would recommend to use the similar  [toReference](https://github.com/generic-datalake/iceberg-poc/pull/2/files#diff-86160616589acf1dd526b10b73418a46fe60f9e5e5ab6946a4ea3c8f019542f5R181) method to check whether it's indeed a `FieldReferenceExpression`,  that's more safe.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r548776839



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -102,4 +105,154 @@ public void testLimitPushDown() {
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testFilterPushDown() {

Review comment:
       I add a listener to validate the pushdown for filter in [TestFlinkTableSource](https://github.com/zhangjun0x01/iceberg/blob/filterPushDown/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java)




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551840539



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -72,8 +72,21 @@ protected TableEnvironment getTableEnv() {
     return tEnv;
   }
 
+  /**
+   * if we have a sql with '%' , for example :  SELECT * FROM mytable WHERE data LIKE 'a%'. the format operation will
+   * throw an exception, so we add a sql method to execute the original sql
+   *
+   * @param query the sql
+   * @param args  the args to format sql
+   * @return the formated sql
+   */
   protected List<Object[]> sql(String query, Object... args) {
-    TableResult tableResult = getTableEnv().executeSql(String.format(query, args));
+    String sql = String.format(query, args);
+    return sql(sql);
+  }
+
+  protected List<Object[]> sql(String sql) {
+    TableResult tableResult = getTableEnv().executeSql(sql);

Review comment:
       I'd prefer to have a basic `args` check here ( rather than introducing another similar `sql(String query, Object... args)` method: 
   
   ```java
   String query = args.length > 0 ? String.format(sql, args) : sql:
   TableResult tableResult = getTableEnv().executeSql(query);
   ```




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556991198



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));

Review comment:
       Actually, since the type is already known there is no need to use `toReference` at all. This can call name directly:
   
   ```java
             return onlyChildAs(call, FieldReferenceExpression.class)
                 .map(FieldReferenceExpression::getName)
                 .map(Expressions::isNull);
   ```




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552379722



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink. the BETWEEN will be convert to (GT_EQ AND
+   * LT_EQ), the NOT_BETWEEN will be convert to (LT_EQ OR GT_EQ), the IN will be convert to OR, so we do not add the
+   * convert here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, call);

Review comment:
       > I think there's a bug here. Assume the case: `a != NaN`, the `handleNaN` will return an iceberg expression: `Expressions.isNaN(name)`. That's incorrect ?
   
   It was my negligence, forgot to deal with `!=`,I updated it




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553710419



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +95,396 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
     Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, mixedResult.size());
+    assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));

Review comment:
       I updated all




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552356432



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, new byte[] {'a', 'b'});
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+
+    matchLiteral("field13", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.equal("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isEqual(Expressions.lit(1)))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isEqual(Expressions.$("field1")))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);

Review comment:
       we can construct `NaN` type by `float('NaN')` in spark and hive sql,
   but I lookup the flink [doc](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/types.html),It does not seem to support `Infinity` an `NaN` now




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552356432



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, new byte[] {'a', 'b'});
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+
+    matchLiteral("field13", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.equal("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isEqual(Expressions.lit(1)))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isEqual(Expressions.$("field1")))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);

Review comment:
       we can construct `NaN` type by `float('NaN')` in spark and hive sql,
   but I lookup the flink [doc](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/types.html),It does not seem to support `Infinity` an `NaN` now




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


[GitHub] [iceberg] zhangjun0x01 commented on pull request #1893: Flink : add filter push down for IcebergTableSource

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-747250046


   @yyanyy thanks for your review,I update all


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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557808135



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",
+            lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("explain should contains the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "a"});
+    expectedGT.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "a"});
+    expectedGTE.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("explain should contains the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("explain should contains the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("should have 1 record", 1, resultNotIn.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)",
+        lastScanEvent.filter().toString());
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "a"});
+    expected.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultNotNull.toArray(), expected.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNull.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT id = 1 ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("should have 2 record", 2, resultNot.size());
+
+    List<Object[]> expectedNot = Lists.newArrayList();
+    expectedNot.add(new Object[] {2, "b"});
+    expectedNot.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNot.toArray(), expectedNot.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "a"});
+    expectedBetween.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultBetween.toArray(), expectedBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)",
+        lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("explain should contains the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'a%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"a\"\"";
+    Assert
+        .assertTrue("the like sql explain should contains the push down filter", explainLike.contains(expectedFilter));
+
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("the like result should produce the expected record", resultLike.get(0),
+        new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // not push down
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%a%' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDown2Literal() {
+    String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME);
+    String explain2Literal = getTableEnv().explainSql(sql2Literal);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explain2Literal.contains(expectedFilterPushDownExplain));
+  }

Review comment:
       We can add the test case `All inequalities with null, like data < null`.
    in spark and hive,we can use sql `select float('NaN')` to get a `NaN` ,but the `NaN` and `Infinity` are not supported by flink ([doc](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/types.html)), we can add a `TODO` in the class to do the test in the futher 




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


[GitHub] [iceberg] openinx commented on pull request #1893: Flink : add filter push down for IcebergTableSource

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-741650219


   Thanks @zhangjun0x01 for contributing,  I will review this patch today or tomorrow. 


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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551889320



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink. the BETWEEN will be convert to (GT_EQ AND

Review comment:
       > the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink
   
   -> `The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically`. 
   
   > so we do not add the convert here
   
   -> so we don not add the conversion here. 
   
   > be convert to
   
   -> be converted to 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink. the BETWEEN will be convert to (GT_EQ AND

Review comment:
       > the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink
   
   -> `The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically`. 
   
   > so we do not add the convert here
   
   -> so we do not add the conversion here. 
   
   > be convert to
   
   -> be converted to 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547964203



##########
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 {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(1);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(0);
+    }
+
+    String name = fieldReferenceExpression.getName();
+    Class clazz = valueLiteralExpression.getOutputDataType().getConversionClass();
+    Object value = valueLiteralExpression.getValueAs(clazz).get();
+
+    BuiltInFunctionDefinition functionDefinition = (BuiltInFunctionDefinition) call.getFunctionDefinition();
+    if (functionDefinition.equals(BuiltInFunctionDefinitions.EQUALS) &&

Review comment:
       Is this sentence correct  ?    should transform the `&&` to `||` ? 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r559001995



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +92,584 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
-    Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
+    Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected records", new Object[] {1, "iceberg", 10.0}, result.get(0));
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    Assert.assertEquals("Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("Should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
-    expectedList.add(new Object[] {1, "a"});
-    expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {1, "iceberg", 10.0});
+    expectedList.add(new Object[] {2, "b", 20.0});
+    expectedList.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected records", expectedList.toArray(), resultExceed.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, mixedResult.size());
+    Assert.assertArrayEquals("Should produce the expected records",
+        new Object[] {1, "iceberg", 10.0}, mixedResult.get(0));
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownEqualNull() {
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlEqualNull);
+    Assert.assertEquals("Should have 0 record", 0, result.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("Should have 1 record", 1, resultLeft.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLeft.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("Should have 2 records", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b", 20.0});
+    expectedNE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedNE.toArray(), resultNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqualNull() {
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+
+    List<Object[]> resultNE = sql(sqlNotEqualNull);
+    Assert.assertEquals("Should have 0 records", 0, resultNE.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"iceberg\"";
+    Assert.assertTrue("Explain should contain the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("Should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultAnd.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("Explain should contain the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("Should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "iceberg", 10.0});
+    expectedOR.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedOR.toArray(), resultOr.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    expectedGT.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanNull() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 records", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "iceberg", 10.0});
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    expectedGTE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualNull() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "iceberg", 10.0});
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanNull() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualNull() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("Explain should contain the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("Should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "iceberg", 10.0});
+    expectedIN.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedIN.toArray(), resultIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownInNull() {
+    String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlInNull);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("Should have 1 record", 1, resultNotIn.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultNotIn.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expectedScan = "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedScan, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotInNull() {
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultGT = sql(sqlNotInNull);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("Should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "iceberg", 10.0});
+    expected.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expected.toArray(), resultNotNull.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("Should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNull.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("Should have 1 record", 1, resultNot.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNot.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("Should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "iceberg", 10.0});
+    expectedBetween.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedBetween.toArray(), resultBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between Should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "iceberg", 10.0});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\"";
+    Assert.assertTrue("the like sql Explain should contain the push down filter", explainLike.contains(expectedFilter));
+
+    sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' ";
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("The like result should produce the expected record",
+        new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterNotPushDownLike() {
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%i' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' ";
+    List<Object[]> resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 0, resultLike.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%i%' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%ice%g' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%%ice%%g' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%%' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 2 records", 2, resultLike.size());
+    List<Object[]> expectedRecords = Lists.newArrayList();
+    expectedRecords.add(new Object[] {1, "iceberg", 10.0});
+    expectedRecords.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedRecords.toArray(), resultLike.toArray());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE 'iceber_' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDown2Literal() {
+    String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME);
+    String explain2Literal = getTableEnv().explainSql(sql2Literal);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explain2Literal.contains(expectedFilterPushDownExplain));
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  /**
+   * NaN is not supported by flink now, so we add the test case to assert the parse error, when we upgrade the flink
+   * that supports NaN, we will delele the method, and add some test case to test NaN.
+   */
+  @Test
+  public void testSqlParseError() {
+    String sqlParseErrorEqual = String.format("SELECT * FROM %s WHERE d = DOUBLE('NaN') ", TABLE_NAME);

Review comment:
       I don't see the function `DOUBLE` documented in Flink's built-in functions. I think what you want is to use `CAST` instead: `CAST('NaN' AS DOUBLE)`.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546046995



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -102,4 +105,154 @@ public void testLimitPushDown() {
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testFilterPushDown() {

Review comment:
       Tests should be broken into individual methods that are each a test case. To share code, use `@Before` and `@After` and different test suites.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557560182



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testNotEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.notEqual(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isNotEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isNotEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testNotEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.notNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve(Expressions.$("field1").isNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve(Expressions.$("field1").isNotNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).and(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    And and = (And) actual.get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), and.op());
+    Assert.assertEquals(expected.left().op(), and.left().op());
+    Assert.assertEquals(expected.right().op(), and.right().op());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).or(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Or or = (Or) actual.get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), or.op());
+    Assert.assertEquals(expected.left().op(), or.left().op());
+    Assert.assertEquals(expected.right().op(), or.right().op());
+  }
+
+  @Test
+  public void testNot() {
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Not not = (Not) actual.get();
+    Not expected = (Not) org.apache.iceberg.expressions.Expressions.not(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1));
+
+    Assert.assertEquals(expected.op(), not.op());
+    assertPredicatesMatch((UnboundPredicate<?>) expected.child(), (UnboundPredicate<?>) not.child());
+  }
+
+  @Test
+  public void testLike() {

Review comment:
       This suite also needs negative tests for `LIKE` with different wildcard locations:
   * The pattern is `"%abc"`
   * The pattern is `"%abc%"`
   * The pattern is `"abc%d"`
   * The pattern is `"%"`




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557561287



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);

Review comment:
       Thank you, these assertions look better now.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557564786



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",
+            lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("explain should contains the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "a"});
+    expectedGT.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "a"});
+    expectedGTE.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("explain should contains the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));

Review comment:
       Needs to check the scan event.




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553154292



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkUtil.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.BoundPredicate;
+import org.apache.iceberg.expressions.ExpressionVisitors;
+import org.apache.iceberg.expressions.Literal;
+import org.apache.iceberg.expressions.UnboundPredicate;
+
+public class FlinkUtil {

Review comment:
       I'd prefer to abstract the common methods from flink and spark to a common utility.   How about introducing a `ExpressionsUtil` under package `org.apache.iceberg.expressions` ?  In that way, we could remove the `Spark3Util#describe` methods. 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551221007



##########
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());

Review comment:
       If we're sure that flink won't enter the `IN` block, then I think we should remove this block. Pls add a comment saying `IN` will convert to multiple `OR`.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558841982



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testNotEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.notEqual(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isNotEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isNotEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testNotEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.notNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve(Expressions.$("field1").isNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve(Expressions.$("field1").isNotNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).and(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    And and = (And) actual.get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), and.op());
+    Assert.assertEquals(expected.left().op(), and.left().op());
+    Assert.assertEquals(expected.right().op(), and.right().op());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).or(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Or or = (Or) actual.get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), or.op());
+    Assert.assertEquals(expected.left().op(), or.left().op());
+    Assert.assertEquals(expected.right().op(), or.right().op());
+  }
+
+  @Test
+  public void testNot() {
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Not not = (Not) actual.get();
+    Not expected = (Not) org.apache.iceberg.expressions.Expressions.not(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1));
+
+    Assert.assertEquals(expected.op(), not.op());
+    assertPredicatesMatch((UnboundPredicate<?>) expected.child(), (UnboundPredicate<?>) not.child());
+  }
+
+  @Test
+  public void testLike() {

Review comment:
       I add the test case ([link](https://github.com/zhangjun0x01/iceberg/blob/filterPushDown/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java#L300)) ,and add the sql test case ([link](https://github.com/zhangjun0x01/iceberg/blob/filterPushDown/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java#L580))




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551900690



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink. the BETWEEN will be convert to (GT_EQ AND
+   * LT_EQ), the NOT_BETWEEN will be convert to (LT_EQ OR GT_EQ), the IN will be convert to OR, so we do not add the
+   * convert here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, call);
+
+        case NOT_EQ:

Review comment:
       Will the flink convert the `NOT_EQ` to be `NOT ( EQ )` ?  If sure, then we don't have to handle the `NOT_EQ` ? 




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557841827



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",
+            lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("explain should contains the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "a"});
+    expectedGT.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "a"});
+    expectedGTE.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("explain should contains the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("explain should contains the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("should have 1 record", 1, resultNotIn.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)",
+        lastScanEvent.filter().toString());
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "a"});
+    expected.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultNotNull.toArray(), expected.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNull.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT id = 1 ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("should have 2 record", 2, resultNot.size());
+
+    List<Object[]> expectedNot = Lists.newArrayList();
+    expectedNot.add(new Object[] {2, "b"});
+    expectedNot.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNot.toArray(), expectedNot.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "a"});
+    expectedBetween.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultBetween.toArray(), expectedBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)",
+        lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("explain should contains the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'a%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"a\"\"";
+    Assert
+        .assertTrue("the like sql explain should contains the push down filter", explainLike.contains(expectedFilter));
+
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("the like result should produce the expected record", resultLike.get(0),
+        new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // not push down
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%a%' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDown2Literal() {
+    String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME);
+    String explain2Literal = getTableEnv().explainSql(sql2Literal);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explain2Literal.contains(expectedFilterPushDownExplain));
+  }

Review comment:
       yes,I will add the test case to assert parse error, when we upgrade to a version of flink that supports `NaN`, we will not forget to modify these test cases that fail.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558841748



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",
+            lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("explain should contains the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "a"});
+    expectedGT.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "a"});
+    expectedGTE.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("explain should contains the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("explain should contains the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("should have 1 record", 1, resultNotIn.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)",
+        lastScanEvent.filter().toString());
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "a"});
+    expected.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultNotNull.toArray(), expected.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNull.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT id = 1 ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("should have 2 record", 2, resultNot.size());
+
+    List<Object[]> expectedNot = Lists.newArrayList();
+    expectedNot.add(new Object[] {2, "b"});
+    expectedNot.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNot.toArray(), expectedNot.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "a"});
+    expectedBetween.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultBetween.toArray(), expectedBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)",
+        lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("explain should contains the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'a%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"a\"\"";
+    Assert
+        .assertTrue("the like sql explain should contains the push down filter", explainLike.contains(expectedFilter));
+
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("the like result should produce the expected record", resultLike.get(0),
+        new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // not push down
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%a%' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDown2Literal() {
+    String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME);
+    String explain2Literal = getTableEnv().explainSql(sql2Literal);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explain2Literal.contains(expectedFilterPushDownExplain));
+  }

Review comment:
       I add the test case ([link](https://github.com/zhangjun0x01/iceberg/blob/filterPushDown/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java#L300)) ,and add the sql test case ([link](https://github.com/zhangjun0x01/iceberg/blob/filterPushDown/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java#L580))




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551986989



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java
##########
@@ -122,6 +135,24 @@ 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);
+  }
+
+  @Override
+  public TableSource<RowData> applyPredicate(List<Expression> predicates) {
+    List<org.apache.iceberg.expressions.Expression> expressions = Lists.newArrayList();
+    for (Expression predicate : predicates) {
+      Optional<org.apache.iceberg.expressions.Expression> expression = FlinkFilters.convert(predicate);
+      if (expression.isPresent()) {
+        expressions.add(expression.get());
+      }
+    }

Review comment:
       nit: how about rewrite those as: 
   
   ```java
       List<org.apache.iceberg.expressions.Expression> expressions = Lists.newArrayList();
       for (Expression predicate : predicates) {
         FlinkFilters.convert(predicate).ifPresent(expressions::add);
       }
   ```




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-762465043


   Thanks, @zhangjun0x01! It is great that this will be in the 0.11.0 release. Thanks for getting it done!


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r559001995



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +92,584 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
-    Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
+    Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected records", new Object[] {1, "iceberg", 10.0}, result.get(0));
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    Assert.assertEquals("Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("Should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
-    expectedList.add(new Object[] {1, "a"});
-    expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {1, "iceberg", 10.0});
+    expectedList.add(new Object[] {2, "b", 20.0});
+    expectedList.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected records", expectedList.toArray(), resultExceed.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, mixedResult.size());
+    Assert.assertArrayEquals("Should produce the expected records",
+        new Object[] {1, "iceberg", 10.0}, mixedResult.get(0));
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownEqualNull() {
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlEqualNull);
+    Assert.assertEquals("Should have 0 record", 0, result.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("Should have 1 record", 1, resultLeft.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLeft.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("Should have 2 records", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b", 20.0});
+    expectedNE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedNE.toArray(), resultNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqualNull() {
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+
+    List<Object[]> resultNE = sql(sqlNotEqualNull);
+    Assert.assertEquals("Should have 0 records", 0, resultNE.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"iceberg\"";
+    Assert.assertTrue("Explain should contain the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("Should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultAnd.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("Explain should contain the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("Should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "iceberg", 10.0});
+    expectedOR.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedOR.toArray(), resultOr.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    expectedGT.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanNull() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 records", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "iceberg", 10.0});
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    expectedGTE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualNull() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "iceberg", 10.0});
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanNull() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualNull() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("Explain should contain the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("Should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "iceberg", 10.0});
+    expectedIN.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedIN.toArray(), resultIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownInNull() {
+    String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlInNull);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("Should have 1 record", 1, resultNotIn.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultNotIn.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expectedScan = "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedScan, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotInNull() {
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultGT = sql(sqlNotInNull);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("Should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "iceberg", 10.0});
+    expected.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expected.toArray(), resultNotNull.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("Should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNull.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("Should have 1 record", 1, resultNot.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNot.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("Should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "iceberg", 10.0});
+    expectedBetween.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedBetween.toArray(), resultBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between Should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "iceberg", 10.0});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\"";
+    Assert.assertTrue("the like sql Explain should contain the push down filter", explainLike.contains(expectedFilter));
+
+    sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' ";
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("The like result should produce the expected record",
+        new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterNotPushDownLike() {
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%i' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' ";
+    List<Object[]> resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 0, resultLike.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%i%' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%ice%g' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%%ice%%g' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%%' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 2 records", 2, resultLike.size());
+    List<Object[]> expectedRecords = Lists.newArrayList();
+    expectedRecords.add(new Object[] {1, "iceberg", 10.0});
+    expectedRecords.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedRecords.toArray(), resultLike.toArray());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE 'iceber_' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDown2Literal() {
+    String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME);
+    String explain2Literal = getTableEnv().explainSql(sql2Literal);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explain2Literal.contains(expectedFilterPushDownExplain));
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  /**
+   * NaN is not supported by flink now, so we add the test case to assert the parse error, when we upgrade the flink
+   * that supports NaN, we will delele the method, and add some test case to test NaN.
+   */
+  @Test
+  public void testSqlParseError() {
+    String sqlParseErrorEqual = String.format("SELECT * FROM %s WHERE d = DOUBLE('NaN') ", TABLE_NAME);

Review comment:
       I don't see the function `DOUBLE` documented in [Flink's built-in functions](https://ci.apache.org/projects/flink/flink-docs-release-1.8/dev/table/functions.html#type-conversion-functions). I think what you want is to use `CAST` instead: `CAST('NaN' AS DOUBLE)`.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556987116



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);
+      }
+
+      return o;
+    });
+  }
+
+  private static boolean literalOnRight(List<ResolvedExpression> args) {
+    return args.get(0) instanceof FieldReferenceExpression && args.get(1) instanceof ValueLiteralExpression;
+  }
+
+  private static Tuple2<String, Object> parseFieldAndLiteral(CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args.size() != 2) {
+      return null;

Review comment:
       Other methods return `Optional`. Why use null here instead?




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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-761290725


   I'd like to get this into the 0.11.0 release, if possible. Thanks for working on this, @zhangjun0x01! It will be great to have this feature done.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557001295



##########
File path: api/src/main/java/org/apache/iceberg/expressions/ExpressionsUtil.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.expressions;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class ExpressionsUtil {

Review comment:
       `Expressions` should not be plural. It should be `ExpressionUtil` instead.




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


[GitHub] [iceberg] rdblue merged pull request #1893: Flink: Support filter pushdown in IcebergTableSource

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893


   


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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-752311714


   > @zhangjun0x01 Would you mind to resolve the conflicts ? I will take another look at this PR if you resolve it. Thanks.
   
   @openinx  I fixed the conflicts


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558991485



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +92,584 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
-    Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
+    Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected records", new Object[] {1, "iceberg", 10.0}, result.get(0));
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    Assert.assertEquals("Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("Should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
-    expectedList.add(new Object[] {1, "a"});
-    expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {1, "iceberg", 10.0});
+    expectedList.add(new Object[] {2, "b", 20.0});
+    expectedList.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected records", expectedList.toArray(), resultExceed.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, mixedResult.size());
+    Assert.assertArrayEquals("Should produce the expected records",
+        new Object[] {1, "iceberg", 10.0}, mixedResult.get(0));
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownEqualNull() {
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlEqualNull);
+    Assert.assertEquals("Should have 0 record", 0, result.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("Should have 1 record", 1, resultLeft.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLeft.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("Should have 2 records", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b", 20.0});
+    expectedNE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedNE.toArray(), resultNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqualNull() {
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+
+    List<Object[]> resultNE = sql(sqlNotEqualNull);
+    Assert.assertEquals("Should have 0 records", 0, resultNE.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"iceberg\"";
+    Assert.assertTrue("Explain should contain the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("Should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultAnd.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("Explain should contain the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("Should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "iceberg", 10.0});
+    expectedOR.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedOR.toArray(), resultOr.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    expectedGT.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanNull() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 records", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "iceberg", 10.0});
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    expectedGTE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualNull() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "iceberg", 10.0});
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanNull() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualNull() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("Explain should contain the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("Should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "iceberg", 10.0});
+    expectedIN.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedIN.toArray(), resultIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownInNull() {
+    String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlInNull);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("Should have 1 record", 1, resultNotIn.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultNotIn.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expectedScan = "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedScan, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotInNull() {
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultGT = sql(sqlNotInNull);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("Should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "iceberg", 10.0});
+    expected.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expected.toArray(), resultNotNull.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("Should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNull.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("Should have 1 record", 1, resultNot.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNot.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("Should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "iceberg", 10.0});
+    expectedBetween.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedBetween.toArray(), resultBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between Should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "iceberg", 10.0});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\"";
+    Assert.assertTrue("the like sql Explain should contain the push down filter", explainLike.contains(expectedFilter));
+
+    sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' ";
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("The like result should produce the expected record",
+        new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterNotPushDownLike() {
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%i' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' ";
+    List<Object[]> resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 0, resultLike.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%i%' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%ice%g' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%%ice%%g' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%%' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 2 records", 2, resultLike.size());
+    List<Object[]> expectedRecords = Lists.newArrayList();
+    expectedRecords.add(new Object[] {1, "iceberg", 10.0});
+    expectedRecords.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedRecords.toArray(), resultLike.toArray());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE 'iceber_' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));

Review comment:
       By the way, these aren't blockers, but it is good to fix them as long as there are other things that need to be fixed, like the unparameterized use of parameterized types.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552380672



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, new byte[] {'a', 'b'});
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+
+    matchLiteral("field13", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.equal("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isEqual(Expressions.lit(1)))).get();

Review comment:
       I changed all `get` to `orElse(null)`, my initial thought was that in this specified case, the result should not return null, so I did not add judgment




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557815608



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -73,15 +73,15 @@ protected TableEnvironment getTableEnv() {
   }
 
   protected static TableResult exec(TableEnvironment env, String query, Object... args) {
-    return env.executeSql(String.format(query, args));
+    return env.executeSql(args.length > 0 ? String.format(query, args) : query);

Review comment:
       You can use `%%` in a format string to produce a literal `%`.
   
   I think it would be better not to change this file, if it isn't necessary.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546049529



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -102,4 +105,154 @@ public void testLimitPushDown() {
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testFilterPushDown() {

Review comment:
       Can you also add a test case that listens for a `ScanEvent` and validates that the expression was correctly passed to Iceberg?




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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546040235



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

Review comment:
       Null values can't be ignored. This should either return `Optional.empty` or throw `IllegalArgumentException` if there is a null value.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551859042



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -72,8 +72,21 @@ protected TableEnvironment getTableEnv() {
     return tEnv;
   }
 
+  /**
+   * if we have a sql with '%' , for example :  SELECT * FROM mytable WHERE data LIKE 'a%'. the format operation will
+   * throw an exception, so we add a sql method to execute the original sql
+   *
+   * @param query the sql
+   * @param args  the args to format sql
+   * @return the formated sql
+   */
   protected List<Object[]> sql(String query, Object... args) {
-    TableResult tableResult = getTableEnv().executeSql(String.format(query, args));
+    String sql = String.format(query, args);
+    return sql(sql);
+  }
+
+  protected List<Object[]> sql(String sql) {
+    TableResult tableResult = getTableEnv().executeSql(sql);

Review comment:
       I update it




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r548782011



##########
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());

Review comment:
       the `IN`,`BETWEEN`,`NOT_BETWEEN` [will be auto convert in flink](
   https://github.com/apache/iceberg/pull/1893#discussion_r548778689)  ,so we will not enter the `IN` block, 
   should we delete `IN` branch ? 




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


[GitHub] [iceberg] zhangjun0x01 commented on pull request #1893: Flink : add filter push down for IcebergTableSource

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-741487016


   @openinx  could you help me review the pr when you have time ,thanks 


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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551903280



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkUtil.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.BoundPredicate;
+import org.apache.iceberg.expressions.ExpressionVisitors;
+import org.apache.iceberg.expressions.Literal;
+import org.apache.iceberg.expressions.UnboundPredicate;
+
+public class FlinkUtil {

Review comment:
       It's good to move this class to `test` packages, I saw nobody from production files would call this. 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551992290



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, new byte[] {'a', 'b'});
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+
+    matchLiteral("field13", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.equal("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isEqual(Expressions.lit(1)))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isEqual(Expressions.$("field1")))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);
+  }
+
+  @Test
+  public void testNotEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.notEqual("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isNotEqual(Expressions.lit(1)))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isNotEqual(Expressions.$("field1")))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1)))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1")))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1)))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1")))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1)))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1")))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1)))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1")))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve($("field1").isNull());
+    UnboundPredicate actual = (UnboundPredicate) FlinkFilters.convert(expr).get();
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve($("field1").isNotNull());
+    UnboundPredicate actual = (UnboundPredicate) FlinkFilters.convert(expr).get();
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, actual);
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(Expressions.$("field1").isEqual(lit(1)).and(Expressions.$("field2").isEqual(lit(2L))));
+    And actual = (And) FlinkFilters.convert(expr).get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    assertEquals(expected.op(), actual.op());
+    assertEquals(expected.left().op(), actual.left().op());
+    assertEquals(expected.right().op(), actual.right().op());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve($("field1").isEqual(lit(1)).or($("field2").isEqual(lit(2L))));
+    Or actual = (Or) FlinkFilters.convert(expr).get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    assertEquals(expected.op(), actual.op());
+    assertEquals(expected.left().op(), actual.left().op());
+    assertEquals(expected.right().op(), actual.right().op());
+  }
+
+  @Test
+  public void testNot() {
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.NOT, $("field1").isEqual(lit(1))));
+    Not actual = (Not) FlinkFilters.convert(expr).get();
+    Not expected = (Not) org.apache.iceberg.expressions.Expressions.not(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1));
+
+    assertEquals(expected.op(), actual.op());
+    assertPredicatesMatch((UnboundPredicate) expected.child(), (UnboundPredicate) actual.child());
+  }
+
+  @Test
+  public void testLike() {
+    UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc");
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.LIKE, $("field5"), lit("abc%")));
+    assertPredicatesMatch(expected, (UnboundPredicate) FlinkFilters.convert(expr).get());
+  }
+
+  private void matchLiteral(String fieldName, Object flinkLiteral, Object icebergLiteral) {
+    Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral)));
+    org.apache.iceberg.expressions.Expression actual = FlinkFilters.convert(expr).get();

Review comment:
       It's good to have a assert the optional is present before get  its value: 
   
   ```java
       Optional<org.apache.iceberg.expressions.Expression> expression = FlinkFilters.convert(expr);
       Assert.assertTrue(expression.isPresent());
   ```




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556977893



##########
File path: api/src/main/java/org/apache/iceberg/expressions/ExpressionsUtil.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.expressions;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class ExpressionsUtil {
+  private ExpressionsUtil() {
+  }
+
+  public static String describe(org.apache.iceberg.expressions.Expression expr) {

Review comment:
       Could you also add Javadoc to this method?




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547673794



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

Review comment:
       I look up the  flink doc and the source code,  and tested it ,it seems that `NaN` and  `Infinity` are not supported by flink now . 
   the data type is supported by flink : [here](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/types.html)
   




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547967889



##########
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:
       I think it's better to add an unit tests to address this case. 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546034840



##########
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();
+                }

Review comment:
       This should not discard anything that is not a `ValueLiteralExpression`. Instead, if there is a non-literal this should either throw `IllegalArgumentException` or return `Optional.empty` to signal that the expression cannot be converted.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558841857



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);

Review comment:
       I updated all




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557016020



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -19,26 +19,36 @@
 
 package org.apache.iceberg.flink;
 
-
 import java.util.List;
 import org.apache.flink.table.api.SqlParserException;
 import org.apache.iceberg.AssertHelpers;
 import org.apache.iceberg.FileFormat;
 import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.events.ScanEvent;
+import org.apache.iceberg.expressions.ExpressionsUtil;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;

Review comment:
       Please remove these imports. The project's style is to use `Assert.assertEquals` and not import static methods in general. This also caused a lot of unnecessary changes.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558991293



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +92,584 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
-    Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
+    Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected records", new Object[] {1, "iceberg", 10.0}, result.get(0));
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    Assert.assertEquals("Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("Should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
-    expectedList.add(new Object[] {1, "a"});
-    expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {1, "iceberg", 10.0});
+    expectedList.add(new Object[] {2, "b", 20.0});
+    expectedList.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected records", expectedList.toArray(), resultExceed.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, mixedResult.size());
+    Assert.assertArrayEquals("Should produce the expected records",
+        new Object[] {1, "iceberg", 10.0}, mixedResult.get(0));
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownEqualNull() {
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlEqualNull);
+    Assert.assertEquals("Should have 0 record", 0, result.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("Should have 1 record", 1, resultLeft.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLeft.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("Should have 2 records", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b", 20.0});
+    expectedNE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedNE.toArray(), resultNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqualNull() {
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+
+    List<Object[]> resultNE = sql(sqlNotEqualNull);
+    Assert.assertEquals("Should have 0 records", 0, resultNE.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"iceberg\"";
+    Assert.assertTrue("Explain should contain the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("Should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultAnd.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("Explain should contain the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("Should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "iceberg", 10.0});
+    expectedOR.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedOR.toArray(), resultOr.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    expectedGT.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanNull() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 records", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "iceberg", 10.0});
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    expectedGTE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualNull() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "iceberg", 10.0});
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanNull() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualNull() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("Explain should contain the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("Should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "iceberg", 10.0});
+    expectedIN.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedIN.toArray(), resultIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownInNull() {
+    String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlInNull);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("Should have 1 record", 1, resultNotIn.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultNotIn.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expectedScan = "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedScan, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotInNull() {
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultGT = sql(sqlNotInNull);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("Should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "iceberg", 10.0});
+    expected.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expected.toArray(), resultNotNull.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("Should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNull.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("Should have 1 record", 1, resultNot.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNot.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("Should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "iceberg", 10.0});
+    expectedBetween.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedBetween.toArray(), resultBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between Should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "iceberg", 10.0});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\"";
+    Assert.assertTrue("the like sql Explain should contain the push down filter", explainLike.contains(expectedFilter));
+
+    sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' ";
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("The like result should produce the expected record",
+        new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterNotPushDownLike() {
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%i' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' ";
+    List<Object[]> resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 0, resultLike.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%i%' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%ice%g' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%%ice%%g' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%%' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 2 records", 2, resultLike.size());
+    List<Object[]> expectedRecords = Lists.newArrayList();
+    expectedRecords.add(new Object[] {1, "iceberg", 10.0});
+    expectedRecords.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedRecords.toArray(), resultLike.toArray());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE 'iceber_' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));

Review comment:
       Style: this line should wrap arguments, not the method call. Can you change these cases to this style?
   
   ```java
       Assert.assertArrayEquals("Should produce the expected record",
           new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
   ```




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557536004



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * Convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          return onlyChildAs(call, FieldReferenceExpression.class)
+              .map(FieldReferenceExpression::getName)
+              .map(Expressions::isNull);
+
+        case NOT_NULL:
+          return onlyChildAs(call, FieldReferenceExpression.class)
+              .map(FieldReferenceExpression::getName)
+              .map(Expressions::notNull);
+
+        case LT:
+          return convertFieldAndLiteral(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertFieldAndLiteral(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertFieldAndLiteral(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertFieldAndLiteral(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return convertFieldAndLiteral((ref, lit) -> {
+            if (NaNUtil.isNaN(lit)) {
+              return Expressions.isNaN(ref);
+            } else {
+              return Expressions.equal(ref, lit);
+            }
+          }, call);
+
+        case NOT_EQ:
+          return convertFieldAndLiteral((ref, lit) -> {
+            if (NaNUtil.isNaN(lit)) {
+              return Expressions.notNaN(ref);
+            } else {
+              return Expressions.notEqual(ref, lit);
+            }
+          }, call);
+
+        case NOT:
+          return onlyChildAs(call, CallExpression.class).flatMap(FlinkFilters::convert).map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> onlyChildAs(CallExpression call,
+                                                                        Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args.size() != 2) {
+      return Optional.empty();
+    }
+
+    org.apache.flink.table.expressions.Expression left = args.get(0);
+    org.apache.flink.table.expressions.Expression right = args.get(1);
+
+    if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) {
+      String name = ((FieldReferenceExpression) left).getName();
+      return convertLiteral((ValueLiteralExpression) right).flatMap(lit -> {
+        if (lit instanceof String) {
+          String pattern = (String) lit;
+          Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+          // exclude special char of LIKE
+          // '_' is the wildcard of the SQL LIKE
+          if (!pattern.contains("_") && matcher.matches()) {
+            return Optional.of(Expressions.startsWith(name, matcher.group(1)));
+          }
+        }
+
+        return Optional.empty();
+      });
+    }
+
+    return Optional.empty();
+  }
+
+  private static Optional<Expression> convertLogicExpression(BiFunction<Expression, Expression, Expression> function,
+                                                             CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args == null || args.size() != 2) {
+      return Optional.empty();
+    }
+
+    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<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);
+      }
+
+      return o;
+    });
+  }
+
+  private static Optional<Expression> convertFieldAndLiteral(BiFunction<String, Object, Expression> expr,
+                                                             CallExpression call) {
+    return convertFieldAndLiteral(expr, expr, call);
+  }
+
+  private static Optional<Expression> convertFieldAndLiteral(
+      BiFunction<String, Object, Expression> convertLR, BiFunction<String, Object, Expression> convertRL,
+      CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args.size() != 2) {
+      return Optional.empty();
+    }
+
+    org.apache.flink.table.expressions.Expression left = args.get(0);
+    org.apache.flink.table.expressions.Expression right = args.get(1);
+
+    if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) {
+      String name = ((FieldReferenceExpression) left).getName();
+      Optional<Object> lit = convertLiteral((ValueLiteralExpression) right);
+      if (lit.isPresent()) {
+        return Optional.of(convertLR.apply(name, lit.get()));

Review comment:
       Why not use `lit.map` instead of putting the `if` here?




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557880023



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -73,15 +73,15 @@ protected TableEnvironment getTableEnv() {
   }
 
   protected static TableResult exec(TableEnvironment env, String query, Object... args) {
-    return env.executeSql(String.format(query, args));
+    return env.executeSql(args.length > 0 ? String.format(query, args) : query);

Review comment:
       ok,I will reverting it 




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553711041



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, new byte[] {'a', 'b'});
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+
+    matchLiteral("field13", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.equal("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isEqual(Expressions.lit(1)))).orElse(null);
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isEqual(Expressions.$("field1")))).orElse(null);
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN)))).orElse(null);
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3")))).orElse(null);
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);
+  }
+
+  @Test
+  public void testNotEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.notEqual("field1", 1);

Review comment:
       I added all type test for `testNotEquals ` and `testEquals `




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552009222



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkUtil.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.BoundPredicate;
+import org.apache.iceberg.expressions.ExpressionVisitors;
+import org.apache.iceberg.expressions.Literal;
+import org.apache.iceberg.expressions.UnboundPredicate;
+
+public class FlinkUtil {

Review comment:
       Sounds like it should be renamed as `FlinkFiltersUtil`




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553160221



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);

Review comment:
       I think the correct way to validate the flink-iceberg type mapping is:  for each data type,  providing a EQUAL expression and then use the `FlinkFilters` convert it to iceberg filters,  finally call the `BoundLiteralPredicate#test` to see the EQUALS return the expected `true`,  if yes, then the data type mapping MUST be correct.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557017512



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  private static final List<Tuple2<String, Object>> LIST = ImmutableList.of(

Review comment:
       Is there a more descriptive name than `LIST`? What are these used for?




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558990584



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.Pair;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Pair<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Pair.of("field1", 1),
+      Pair.of("field2", 2L),
+      Pair.of("field3", 3F),
+      Pair.of("field4", 4D),
+      Pair.of("field5", "iceberg"),
+      Pair.of("field6", true),
+      Pair.of("field7", new byte[] {'a', 'b'}),
+      Pair.of("field8", BigDecimal.valueOf(10.12)),
+      Pair.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Pair.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Pair.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Pair.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10.12), BigDecimal.valueOf(10.12));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Pair<String, Object> pair : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(pair.first(), pair.second());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(pair.first()).isEqual(Expressions.lit(pair.second()))));
+      Assert.assertTrue("Conversion should succeed", actual.isPresent());
+      assertPredicatesMatch(expected, actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(pair.second()).isEqual(Expressions.$(pair.first()))));
+      Assert.assertTrue("Conversion should succeed", actual1.isPresent());
+      assertPredicatesMatch(expected, actual1.get());
+    }
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    assertPredicatesMatch(expected, actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3"))));
+    Assert.assertTrue("Conversion should succeed", actual1.isPresent());
+    assertPredicatesMatch(expected, actual1.get());
+  }
+
+  @Test
+  public void testNotEquals() {
+    for (Pair<String, Object> pair : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.notEqual(pair.first(), pair.second());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(pair.first()).isNotEqual(Expressions.lit(pair.second()))));
+      Assert.assertTrue("Conversion should succeed", actual.isPresent());
+      assertPredicatesMatch(expected, actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(pair.second()).isNotEqual(Expressions.$(pair.first()))));
+      Assert.assertTrue("Conversion should succeed", actual1.isPresent());
+      assertPredicatesMatch(expected, actual1.get());
+    }
+  }
+
+  @Test
+  public void testNotEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.notNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    assertPredicatesMatch(expected, actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3"))));
+    Assert.assertTrue("Conversion should succeed", actual1.isPresent());
+    assertPredicatesMatch(expected, actual1.get());
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1))));
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    assertPredicatesMatch(expected, actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1"))));
+    Assert.assertTrue("Conversion should succeed", actual1.isPresent());
+    assertPredicatesMatch(expected, actual1.get());
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1))));
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    assertPredicatesMatch(expected, actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue("Conversion should succeed", actual1.isPresent());
+    assertPredicatesMatch(expected, actual1.get());
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1))));
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    assertPredicatesMatch(expected, actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1"))));
+    Assert.assertTrue("Conversion should succeed", actual1.isPresent());
+    assertPredicatesMatch(expected, actual1.get());
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1))));
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    assertPredicatesMatch(expected, actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue("Conversion should succeed", actual1.isPresent());
+    assertPredicatesMatch(expected, actual1.get());
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve(Expressions.$("field1").isNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, actual.get());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve(Expressions.$("field1").isNotNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, actual.get());
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).and(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    And and = (And) actual.get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    assertPredicatesMatch(expected.left(), and.left());
+    assertPredicatesMatch(expected.right(), and.right());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).or(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    Or or = (Or) actual.get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    assertPredicatesMatch(expected.left(), or.left());
+    assertPredicatesMatch(expected.right(), or.right());
+  }
+
+  @Test
+  public void testNot() {
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue("Conversion should succeed", actual.isPresent());
+    Not not = (Not) actual.get();
+    Not expected = (Not) org.apache.iceberg.expressions.Expressions.not(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1));
+
+    Assert.assertEquals("Predicate operation should match", expected.op(), not.op());
+    assertPredicatesMatch((UnboundPredicate<?>) expected.child(), (UnboundPredicate<?>) not.child());

Review comment:
       These casts are unnecessary.




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547985792



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -102,4 +105,154 @@ public void testLimitPushDown() {
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testFilterPushDown() {
+    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b'),(3,CAST(null AS VARCHAR))", TABLE_NAME);
+
+    String expectedExplain = "FilterPushDown";
+
+    // not push down
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%a%' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    assertFalse("explain should not contains FilterPushDown", explainNoPushDown.contains(expectedExplain));
+
+    // equal
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    assertTrue("explain should contains FilterPushDown", explain.contains(expectedExplain));

Review comment:
       I will suggest to have a more strict assertion to validate the pushed  filter's internal information, see https://github.com/generic-datalake/iceberg-poc/pull/2/files#diff-5d18d1ff127d1dc70a9a15bbe941f2b6f9d28b3015924f601ac1f722914099dbR96




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546892933



##########
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 {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(1);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(0);
+    }
+
+    String name = fieldReferenceExpression.getName();
+    Class clazz = valueLiteralExpression.getOutputDataType().getConversionClass();
+    Object value = valueLiteralExpression.getValueAs(clazz).get();

Review comment:
       I think it would be good to fix this case rather than making the assumption that Flink won't push the `= null` filter. Handling null will be good for maintainability.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546046202



##########
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 {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(1);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(0);
+    }
+
+    String name = fieldReferenceExpression.getName();
+    Class clazz = valueLiteralExpression.getOutputDataType().getConversionClass();
+    Object value = valueLiteralExpression.getValueAs(clazz).get();
+
+    BuiltInFunctionDefinition functionDefinition = (BuiltInFunctionDefinition) call.getFunctionDefinition();
+    if (functionDefinition.equals(BuiltInFunctionDefinitions.EQUALS) &&
+        functionDefinition.equals(BuiltInFunctionDefinitions.NOT_EQUALS)) {
+      Preconditions.checkNotNull(value, "Expression is always false : %s", call);
+      if (NaNUtil.isNaN(value)) {
+        return Optional.of(Expressions.isNaN(name));
+      } else {
+        return Optional.of(function.apply(name, value));
+      }
+    }
+
+    return Optional.of(function.apply(name, value));

Review comment:
       The literal value needs to be converted to Iceberg's internal representation before being passed to create an expression. Flink will return `LocalDate`, `LocalTime`, `LocalDateTime`, etc. just in the `getValueAs` method. And it isn't clear whether the value stored in the literal is the correct representation for other types as well.
   
   @openinx, could you help recommend how to do the conversion here?




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551697021



##########
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());

Review comment:
       I remove `IN` block and add some comments




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551899740



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN,IN expression will be auto convert by flink. the BETWEEN will be convert to (GT_EQ AND
+   * LT_EQ), the NOT_BETWEEN will be convert to (LT_EQ OR GT_EQ), the IN will be convert to OR, so we do not add the
+   * convert here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, call);

Review comment:
       Pls add an unit test for this if possible. 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557537824



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java
##########
@@ -112,6 +120,10 @@ public String explainSource() {
       explain += String.format(", LimitPushDown : %d", limit);
     }
 
+    if (isFilterPushedDown()) {
+      explain += String.format(", FilterPushDown,the filters :%s", Joiner.on(",").join(filters));

Review comment:
       Nit: typos in this message. There should be a space after `FilterPushDown,` and the space after `filters` should be after the `:`, not before it. The Joiner should be on `", "` to separate the filters, and it should be a private static final constant.




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r552002759



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, new byte[] {'a', 'b'});
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+
+    matchLiteral("field13", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.equal("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isEqual(Expressions.lit(1)))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isEqual(Expressions.$("field1")))).get();
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);

Review comment:
       Pls add `NaN` cases for both equals and notEquals . 




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r548778689



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

Review comment:
       I tested that ,I found `BETWEEN`, `NOT_BETWEEN`,`IN` expression will be auto convert. 
   the `BETWEEN` will be convert to `(GT_EQ AND LT_EQ)`,
   the `NOT_BETWEEN` will be convert to `(LT_EQ OR GT_EQ)`,
   the `IN` will be convert to `OR`,
   
   in `IcebergTableSource#applyPredicate` method ,We won't get `BETWEEN`,`NOT_BETWEEN`,`IN` expressions,so I do not add `BETWEEN`,`NOT_BETWEEN` in `FlinkFilters`, and do not add test case in `TestFlinkFilters`.but add test  case in `TestFlinkTableSource` to valiate the convert.
   
   for example , we have a sql `SELECT * FROM table WHERE id BETWEEN 1 AND 2`,the filter expression will be `id >= 1 AND id <= 2` , the test case is [here](https://github.com/zhangjun0x01/iceberg/blob/filterPushDown/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java#L364) 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553980299



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);

Review comment:
       OK,  that make sense. 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557000212



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * Convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);
+      }
+
+      return o;
+    });
+  }
+
+  private static boolean literalOnRight(List<ResolvedExpression> args) {
+    return args.get(0) instanceof FieldReferenceExpression && args.get(1) instanceof ValueLiteralExpression;
+  }
+
+  private static Tuple2<String, Object> parseFieldAndLiteral(CallExpression call) {

Review comment:
       I think that this class can be cleaned up by rewriting this method. I think this was intended to make reuse easier, but rewriting this allows removing several methods (`literalOnRight`, `convertComparisonExpression`, `handleNaN`, `toReference`, and `toLiteral`), gets rid of the strange method contract where `Tuple2` and `null` are used, and also ensures that `instanceof` checks are done in just one place, rather than with multiple calls to `literalOnRight`.
   
   Here's what I rewrote it to:
   
   ```java
     private static Optional<Expression> convertFieldAndLiteral(BiFunction<String, Object, Expression> expr,
                                                                CallExpression call) {
       return convertFieldAndLiteral(expr, expr, call);
     }
   
     private static Optional<Expression> convertFieldAndLiteral(
         BiFunction<String, Object, Expression> convertLR, BiFunction<String, Object, Expression> convertRL,
         CallExpression call) {
       List<ResolvedExpression> args = call.getResolvedChildren();
       if (args.size() != 2) {
         return Optional.empty();
       }
   
       org.apache.flink.table.expressions.Expression left = args.get(0);
       org.apache.flink.table.expressions.Expression right = args.get(1);
   
       if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) {
         String name = ((FieldReferenceExpression) left).getName();
         Object lit = convertLiteral((ValueLiteralExpression) right);
         return Optional.of(convertLR.apply(name, lit));
   
       } else if (left instanceof ValueLiteralExpression && right instanceof FieldReferenceExpression) {
         Object lit = convertLiteral((ValueLiteralExpression) left);
         String name = ((FieldReferenceExpression) right).getName();
         return Optional.of(convertRL.apply(name, lit));
       }
   
       return Optional.empty();
     }
   ```
   
   The comparison cases in the `switch` statement can then call this directly:
   
   ```java
         switch (op) {
           case LT:
             return convertFieldAndLiteral(Expressions::lessThan, Expressions::greaterThan, call);
   
           case LT_EQ:
             return convertFieldAndLiteral(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
   
           case GT:
             return convertFieldAndLiteral(Expressions::greaterThan, Expressions::lessThan, call);
   
           case GT_EQ:
             return convertFieldAndLiteral(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
   
           case EQ:
             return convertFieldAndLiteral((ref, lit) -> {
               if (NaNUtil.isNaN(lit)) {
                 return Expressions.isNaN(ref);
               } else {
                 return Expressions.equal(ref, lit);
               }
             }, call);
   
           case NOT_EQ:
             return convertFieldAndLiteral((ref, lit) -> {
               if (NaNUtil.isNaN(lit)) {
                 return Expressions.notNaN(ref);
               } else {
                 return Expressions.notEqual(ref, lit);
               }
             }, call);
   ```
   
   The only problem is that `convertLike` can't call this. But that's okay because I don't think the reversed form of `LIKE` is actually supported anyway (`"a%" LIKE col` is not allowed, right?).
   
   So I also rewrote `convertLike`:
   ```java
     private static Optional<Expression> convertLike(CallExpression call) {
       List<ResolvedExpression> args = call.getResolvedChildren();
       if (args.size() != 2) {
         return Optional.empty();
       }
   
       org.apache.flink.table.expressions.Expression left = args.get(0);
       org.apache.flink.table.expressions.Expression right = args.get(1);
   
       if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) {
         String name = ((FieldReferenceExpression) left).getName();
         return convertLiteral((ValueLiteralExpression) right).flatMap(lit -> {
           if (lit instanceof String) {
             String pattern = (String) lit;
             Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
             // exclude special char of LIKE
             // '_' is the wildcard of the SQL LIKE
             if (!pattern.contains("_") && matcher.matches()) {
               return Optional.of(Expressions.startsWith(name, matcher.group(1)));
             }
           }
   
           return Optional.empty();
         });
       }
   
       return Optional.empty();
     }
   ```




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-756933941


   I'll take another look.


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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r558990475



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.equal(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testNotEquals() {
+    for (Tuple2<String, Object> tuple2 : FIELD_VALUE_LIST) {
+      UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.notEqual(tuple2.f0, tuple2.f1);
+
+      Optional<org.apache.iceberg.expressions.Expression> actual =
+          FlinkFilters.convert(resolve(Expressions.$(tuple2.f0).isNotEqual(Expressions.lit(tuple2.f1))));
+      Assert.assertTrue(actual.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+      Optional<org.apache.iceberg.expressions.Expression> actual1 =
+          FlinkFilters.convert(resolve(Expressions.lit(tuple2.f1).isNotEqual(Expressions.$(tuple2.f0))));
+      Assert.assertTrue(actual1.isPresent());
+      assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+    }
+  }
+
+  @Test
+  public void testNotEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.notNaN("field3");
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testGreaterThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThan() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThan("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testLessThanEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1);
+
+    Optional<org.apache.iceberg.expressions.Expression> actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1))));
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+
+    Optional<org.apache.iceberg.expressions.Expression> actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1"))));
+    Assert.assertTrue(actual1.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual1.get());
+  }
+
+  @Test
+  public void testIsNull() {
+    Expression expr = resolve(Expressions.$("field1").isNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.isNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testIsNotNull() {
+    Expression expr = resolve(Expressions.$("field1").isNotNull());
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    UnboundPredicate<Object> expected = org.apache.iceberg.expressions.Expressions.notNull("field1");
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  @Test
+  public void testAnd() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).and(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    And and = (And) actual.get();
+    And expected = (And) org.apache.iceberg.expressions.Expressions.and(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), and.op());
+    Assert.assertEquals(expected.left().op(), and.left().op());
+    Assert.assertEquals(expected.right().op(), and.right().op());
+  }
+
+  @Test
+  public void testOr() {
+    Expression expr = resolve(
+        Expressions.$("field1").isEqual(Expressions.lit(1)).or(Expressions.$("field2").isEqual(Expressions.lit(2L))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Or or = (Or) actual.get();
+    Or expected = (Or) org.apache.iceberg.expressions.Expressions.or(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1),
+        org.apache.iceberg.expressions.Expressions.equal("field2", 2L));
+
+    Assert.assertEquals(expected.op(), or.op());
+    Assert.assertEquals(expected.left().op(), or.left().op());
+    Assert.assertEquals(expected.right().op(), or.right().op());
+  }
+
+  @Test
+  public void testNot() {
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1))));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    Not not = (Not) actual.get();
+    Not expected = (Not) org.apache.iceberg.expressions.Expressions.not(
+        org.apache.iceberg.expressions.Expressions.equal("field1", 1));
+
+    Assert.assertEquals(expected.op(), not.op());
+    assertPredicatesMatch((UnboundPredicate<?>) expected.child(), (UnboundPredicate<?>) not.child());
+  }
+
+  @Test
+  public void testLike() {
+    UnboundPredicate<?> expected = org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc");
+    Expression expr = resolve(ApiExpressionUtils.unresolvedCall(
+        BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%")));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+    assertPredicatesMatch(expected, (UnboundPredicate<?>) actual.get());
+  }
+
+  private void matchLiteral(String fieldName, Object flinkLiteral, Object icebergLiteral) {
+    Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral)));
+    Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
+    Assert.assertTrue(actual.isPresent());
+
+    BoundLiteralPredicate predicate =

Review comment:
       To solve that problem, you can parameterize the method, like this:
   
   ```java
     @SuppressWarnings("unchecked")
     private <T> void matchLiteral(String fieldName, Object flinkLiteral, T icebergLiteral) {
       Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral)));
       Optional<org.apache.iceberg.expressions.Expression> actual = FlinkFilters.convert(expr);
       Assert.assertTrue("Conversion should succeed", actual.isPresent());
   
       BoundLiteralPredicate<T> predicate = (BoundLiteralPredicate<T>)
           ((UnboundPredicate<T>) actual.get()).bind(FlinkSchemaUtil.convert(TABLE_SCHEMA).asStruct(), false);
       Assert.assertTrue(predicate.test(icebergLiteral));
     }
   ```




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551212546



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  public static Optional<Expression> convert(org.apache.flink.table.expressions.Expression flinkExpression) {
+    if (flinkExpression == null || !(flinkExpression instanceof CallExpression)) {
+      return Optional.empty();
+    }
+
+    CallExpression call = (CallExpression) flinkExpression;
+    Operation op = FILTERS.get(call.getFunctionDefinition());
+    if (op != null) {
+      switch (op) {
+        case IS_NULL:
+          Optional<String> name = toReference(getOnlyChild(call, FieldReferenceExpression.class).orElse(null));

Review comment:
       `getOnlyChild` ?  the method name is confusing.   How about using `singleton` ? 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557000909



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * Convert flink expression to iceberg expression.
+   * <p>
+   * the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);
+      }
+
+      return o;
+    });
+  }
+
+  private static boolean literalOnRight(List<ResolvedExpression> args) {
+    return args.get(0) instanceof FieldReferenceExpression && args.get(1) instanceof ValueLiteralExpression;
+  }
+
+  private static Tuple2<String, Object> parseFieldAndLiteral(CallExpression call) {

Review comment:
       @zhangjun0x01, I've posted my rewrite of this class here so you don't have to apply all the changes: https://gist.github.com/rdblue/486bd03df873a2094305025920f94498




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553153180



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkUtil.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.BoundPredicate;
+import org.apache.iceberg.expressions.ExpressionVisitors;
+import org.apache.iceberg.expressions.Literal;
+import org.apache.iceberg.expressions.UnboundPredicate;
+
+public class FlinkUtil {

Review comment:
       The `DescribeExpressionVisitor`  is not actually format the flink filters to string,   instead it's formatting the iceberg's `Expression` to string.  So even if we want to introduce a flink utility ,  it's not a good idea to put the iceberg's visitor in this class. 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556987036



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);
+      }
+
+      return o;
+    });
+  }
+
+  private static boolean literalOnRight(List<ResolvedExpression> args) {
+    return args.get(0) instanceof FieldReferenceExpression && args.get(1) instanceof ValueLiteralExpression;
+  }
+
+  private static Tuple2<String, Object> parseFieldAndLiteral(CallExpression call) {

Review comment:
       I think this should use Iceberg's `Pair` instead of `Tuple2`.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);
+      }
+
+      return o;
+    });
+  }
+
+  private static boolean literalOnRight(List<ResolvedExpression> args) {
+    return args.get(0) instanceof FieldReferenceExpression && args.get(1) instanceof ValueLiteralExpression;
+  }
+
+  private static Tuple2<String, Object> parseFieldAndLiteral(CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    if (args.size() != 2) {
+      return null;

Review comment:
       Other methods return `Optional`. Why use null here instead?




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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r545557481



##########
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:
       thanks for your suggestion,I update it to Optional.
   and I add a not push down test case which return a `Optional.empty()`




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557561853



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));

Review comment:
       This should also test that the scan event had no filter if a scan event was created.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r548067268



##########
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:
       I agree. It is best to validate both child types in each case and only convert if they match what is expected.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r559171436



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +92,584 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
-    Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
+    Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected records", new Object[] {1, "iceberg", 10.0}, result.get(0));
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    Assert.assertEquals("Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("Should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
-    expectedList.add(new Object[] {1, "a"});
-    expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {1, "iceberg", 10.0});
+    expectedList.add(new Object[] {2, "b", 20.0});
+    expectedList.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected records", expectedList.toArray(), resultExceed.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, mixedResult.size());
+    Assert.assertArrayEquals("Should produce the expected records",
+        new Object[] {1, "iceberg", 10.0}, mixedResult.get(0));
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownEqualNull() {
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlEqualNull);
+    Assert.assertEquals("Should have 0 record", 0, result.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("Should have 1 record", 1, resultLeft.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLeft.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("Should have 2 records", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b", 20.0});
+    expectedNE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedNE.toArray(), resultNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqualNull() {
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+
+    List<Object[]> resultNE = sql(sqlNotEqualNull);
+    Assert.assertEquals("Should have 0 records", 0, resultNE.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"iceberg\"";
+    Assert.assertTrue("Explain should contain the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("Should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultAnd.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("Explain should contain the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("Should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "iceberg", 10.0});
+    expectedOR.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedOR.toArray(), resultOr.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    expectedGT.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanNull() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 records", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "iceberg", 10.0});
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    expectedGTE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualNull() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "iceberg", 10.0});
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanNull() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualNull() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("Explain should contain the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("Should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "iceberg", 10.0});
+    expectedIN.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedIN.toArray(), resultIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownInNull() {
+    String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlInNull);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("Should have 1 record", 1, resultNotIn.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultNotIn.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expectedScan = "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedScan, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotInNull() {
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultGT = sql(sqlNotInNull);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("Should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "iceberg", 10.0});
+    expected.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expected.toArray(), resultNotNull.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("Should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNull.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("Should have 1 record", 1, resultNot.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNot.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("Should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "iceberg", 10.0});
+    expectedBetween.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedBetween.toArray(), resultBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between Should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "iceberg", 10.0});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\"";
+    Assert.assertTrue("the like sql Explain should contain the push down filter", explainLike.contains(expectedFilter));
+
+    sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' ";
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("The like result should produce the expected record",
+        new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterNotPushDownLike() {
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%i' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' ";
+    List<Object[]> resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 0, resultLike.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%i%' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%ice%g' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%%ice%%g' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE '%%' ";
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 2 records", 2, resultLike.size());
+    List<Object[]> expectedRecords = Lists.newArrayList();
+    expectedRecords.add(new Object[] {1, "iceberg", 10.0});
+    expectedRecords.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedRecords.toArray(), resultLike.toArray());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+
+    sqlNoPushDown = "SELECT * FROM  " + TABLE_NAME + "  WHERE data LIKE 'iceber_' ";
+    explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+    resultLike = sql(sqlNoPushDown);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLike.get(0));

Review comment:
       I updated all




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-761613165


   @zhangjun0x01, there are still a few things to fix in the tests, mostly minor. But I also found a major problem, which is that `TestFlinkTableSource` now takes a very long time to run. The problem is that the tests run for each format and for 3 different catalog configurations. That means each test runs 9 times and because it is a test that actually runs SQL it takes a long time. The whole suite takes much longer than needed; on my machine, it took 20 minutes!
   
   The filter pushdown tests only need to run for one catalog and one file format because the purpose of those tests is to validate the assumptions of the `FlinkFilter` class with real filters from Flink SQL. The file format and catalog are orthogonal and we don't need to test each one of them. Can you change the parameterization to run with only Avro and a single catalog case?


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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553980299



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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, BiFunction<String, Object, Expression> reversedFunction,
+      CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 != null) {
+      if (literalOnRight(call.getResolvedChildren())) {
+        return Optional.of(function.apply(tuple2.f0, tuple2.f1));
+      } else {
+        return Optional.of(reversedFunction.apply(tuple2.f0, tuple2.f1));
+      }
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  private static Optional<String> toReference(org.apache.flink.table.expressions.Expression expression) {
+    return expression instanceof FieldReferenceExpression ?
+        Optional.of(((FieldReferenceExpression) expression).getName()) :
+        Optional.empty();
+  }
+
+  private static Optional<Object> toLiteral(org.apache.flink.table.expressions.Expression expression) {
+    // Not support null literal
+    return expression instanceof ValueLiteralExpression ?
+        convertLiteral((ValueLiteralExpression) expression) :
+        Optional.empty();
+  }
+
+  private static Optional<Expression> handleNaN(BiFunction<String, Object, Expression> function,
+                                                Function<String, Expression> functionNaN,
+                                                CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String name = tuple2.f0;
+    Object value = tuple2.f1;
+
+    if (NaNUtil.isNaN(value)) {
+      return Optional.of(functionNaN.apply(name));
+    } else {
+      return Optional.of(function.apply(name, value));
+    }
+  }
+
+  private static Optional<Object> convertLiteral(ValueLiteralExpression expression) {
+    Optional<?> value = expression.getValueAs(expression.getOutputDataType().getLogicalType().getDefaultConversion());
+    return value.map(o -> {
+      if (o instanceof LocalDateTime) {
+        return DateTimeUtil.microsFromTimestamp((LocalDateTime) o);
+      } else if (o instanceof Instant) {
+        return DateTimeUtil.microsFromInstant((Instant) o);
+      } else if (o instanceof LocalTime) {
+        return DateTimeUtil.microsFromTime((LocalTime) o);
+      } else if (o instanceof LocalDate) {
+        return DateTimeUtil.daysFromDate((LocalDate) o);

Review comment:
       OK,  that make sense. 




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557808135



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",
+            lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("explain should contains the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "a"});
+    expectedGT.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "a"});
+    expectedGTE.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("explain should contains the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("explain should contains the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("should have 1 record", 1, resultNotIn.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)",
+        lastScanEvent.filter().toString());
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "a"});
+    expected.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultNotNull.toArray(), expected.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNull.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT id = 1 ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("should have 2 record", 2, resultNot.size());
+
+    List<Object[]> expectedNot = Lists.newArrayList();
+    expectedNot.add(new Object[] {2, "b"});
+    expectedNot.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNot.toArray(), expectedNot.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "a"});
+    expectedBetween.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultBetween.toArray(), expectedBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)",
+        lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("explain should contains the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'a%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"a\"\"";
+    Assert
+        .assertTrue("the like sql explain should contains the push down filter", explainLike.contains(expectedFilter));
+
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("the like result should produce the expected record", resultLike.get(0),
+        new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // not push down
+    String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%a%' ";
+    String explainNoPushDown = getTableEnv().explainSql(sqlNoPushDown);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNoPushDown.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDown2Literal() {
+    String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME);
+    String explain2Literal = getTableEnv().explainSql(sql2Literal);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explain2Literal.contains(expectedFilterPushDownExplain));
+  }

Review comment:
       We can add the test case `All inequalities with null, like data < null`.
    in spark and hive,we can use sql `select float('NaN')` to get a `NaN` ,but the `NaN` and `Infinity` are not supported by flink now ([doc](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/types.html)), we can add a `TODO` in the class to do the test in the futher 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557537824



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java
##########
@@ -112,6 +120,10 @@ public String explainSource() {
       explain += String.format(", LimitPushDown : %d", limit);
     }
 
+    if (isFilterPushedDown()) {
+      explain += String.format(", FilterPushDown,the filters :%s", Joiner.on(",").join(filters));

Review comment:
       Nit: typos in this message. Can you change the string to `", FilterPushDown: %s"`? Also, the joiner should be a static final constant and should use `", "` to separate the filters.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551697021



##########
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());

Review comment:
       I remove `IN` block




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556996219



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    return Optional.empty();
+  }
+
+  private static Optional<Expression> convertLogicExpression(BiFunction<Expression, Expression, Expression> function,
+                                                             CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();

Review comment:
       If we enter this method, it means that it is a legal logic expression,  if we add a judgment, it is indeed safer




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557566354



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'a' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"a\"";
+    Assert.assertTrue("explain should contains the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultAnd.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert
+        .assertEquals("should contains the push down filter", "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"a\")",
+            lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("explain should contains the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "a"});
+    expectedOR.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultOr.toArray(), expectedOR.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b"});
+    expectedGT.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("explain should contains the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "a"});
+    expectedGT.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGT.toArray(), expectedGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b"});
+    expectedGTE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("explain should contains the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "a"});
+    expectedGTE.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultGTE.toArray(), expectedGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("explain should contains the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLT.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("explain should contains the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLTE.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("explain should contains the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "a"});
+    expectedIN.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultIN.toArray(), expectedIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // in with null will not push down
+    String sqlInNull = String.format("SELECT * FROM %s WHERE id IN (1,2,NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("explain should contains the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("should have 1 record", 1, resultNotIn.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNotIn.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)",
+        lastScanEvent.filter().toString());
+
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "a"});
+    expected.add(new Object[] {2, "b"});
+    Assert.assertArrayEquals("Should produce the expected record", resultNotNull.toArray(), expected.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("explain should contains the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultNull.get(0), new Object[] {3, null});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNot() {

Review comment:
       It looks like Flink is rewriting the expression to `notEq`. Is there a test case where the `not` is passed to Iceberg? Maybe using a complex expression like `not(or(eq("id", 1), eq("id", 2)))`




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546036726



##########
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());

Review comment:
       I think this should be converted to `List<ValueLiteralExpression>` to simplify value conversion.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r548786358



##########
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()));

Review comment:
       I create a flink issue to track this https://issues.apache.org/jira/browse/FLINK-20767




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546043536



##########
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()));

Review comment:
       This can be `child.map(Expressions::not)`.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546031956



##########
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 {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(1);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(0);
+    }
+
+    String name = fieldReferenceExpression.getName();
+    Class clazz = valueLiteralExpression.getOutputDataType().getConversionClass();
+    Object value = valueLiteralExpression.getValueAs(clazz).get();

Review comment:
       Actually, a few lines down there is an assertion that the value isn't null. This looks like a bug to me.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r559171629



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +92,584 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
-    Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
+    Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected records", new Object[] {1, "iceberg", 10.0}, result.get(0));
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    Assert.assertEquals("Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("Should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
-    expectedList.add(new Object[] {1, "a"});
-    expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {1, "iceberg", 10.0});
+    expectedList.add(new Object[] {2, "b", 20.0});
+    expectedList.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected records", expectedList.toArray(), resultExceed.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    Assert.assertEquals("Should have 1 record", 1, mixedResult.size());
+    Assert.assertArrayEquals("Should produce the expected records",
+        new Object[] {1, "iceberg", 10.0}, mixedResult.get(0));
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownEqualNull() {
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlEqualNull);
+    Assert.assertEquals("Should have 0 record", 0, result.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("Should have 1 record", 1, resultLeft.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLeft.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("Should have 2 records", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b", 20.0});
+    expectedNE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedNE.toArray(), resultNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNoEqualNull() {
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));
+
+    List<Object[]> resultNE = sql(sqlNotEqualNull);
+    Assert.assertEquals("Should have 0 records", 0, resultNE.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownAnd() {
+    String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME);
+    String explainAnd = getTableEnv().explainSql(sqlAnd);
+    String expectedFilter = "ref(name=\"id\") == 1,ref(name=\"data\") == \"iceberg\"";
+    Assert.assertTrue("Explain should contain the push down filter", explainAnd.contains(expectedFilter));
+
+    List<Object[]> resultAnd = sql(sqlAnd);
+    Assert.assertEquals("Should have 1 record", 1, resultAnd.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultAnd.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownOr() {
+    String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME);
+    String explainOr = getTableEnv().explainSql(sqlOr);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")";
+    Assert.assertTrue("Explain should contain the push down filter", explainOr.contains(expectedFilter));
+
+    List<Object[]> resultOr = sql(sqlOr);
+    Assert.assertEquals("Should have 2 record", 2, resultOr.size());
+
+    List<Object[]> expectedOR = Lists.newArrayList();
+    expectedOR.add(new Object[] {1, "iceberg", 10.0});
+    expectedOR.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedOR.toArray(), resultOr.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThan() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") > 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 record", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    expectedGT.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanNull() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanLiteralOnLeft() {
+    String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME);
+    String explainGT = getTableEnv().explainSql(sqlGT);
+    String expectedFilter = "ref(name=\"id\") < 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainGT.contains(expectedFilter));
+
+    List<Object[]> resultGT = sql(sqlGT);
+    Assert.assertEquals("Should have 2 records", 2, resultGT.size());
+
+    List<Object[]> expectedGT = Lists.newArrayList();
+    expectedGT.add(new Object[] {1, "iceberg", 10.0});
+    expectedGT.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGT.toArray(), resultGT.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqual() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") >= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    expectedGTE.add(new Object[] {3, null, 30.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualNull() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainGTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlGTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
+    String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME);
+    String explainGTE = getTableEnv().explainSql(sqlGTE);
+    String expectedFilter = "ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainGTE.contains(expectedFilter));
+
+    List<Object[]> resultGTE = sql(sqlGTE);
+    Assert.assertEquals("Should have 2 records", 2, resultGTE.size());
+
+    List<Object[]> expectedGTE = Lists.newArrayList();
+    expectedGTE.add(new Object[] {1, "iceberg", 10.0});
+    expectedGTE.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedGTE.toArray(), resultGTE.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThan() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") < 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanNull() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLT.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLT);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanLiteralOnLeft() {
+    String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME);
+    String explainLT = getTableEnv().explainSql(sqlLT);
+    String expectedFilter = "ref(name=\"id\") > 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainLT.contains(expectedFilter));
+
+    List<Object[]> resultLT = sql(sqlLT);
+    Assert.assertEquals("Should have 1 record", 1, resultLT.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLT.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqual() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") <= 1";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualNull() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    Assert.assertFalse("Explain should not contain FilterPushDown", explainLTE.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> resultGT = sql(sqlLTE);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertNull("Should not push down a filter", lastScanEvent);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanEqualLiteralOnLeft() {
+    String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id  ", TABLE_NAME);
+    String explainLTE = getTableEnv().explainSql(sqlLTE);
+    String expectedFilter = "ref(name=\"id\") >= 3";
+    Assert.assertTrue("Explain should contain the push down filter", explainLTE.contains(expectedFilter));
+
+    List<Object[]> resultLTE = sql(sqlLTE);
+    Assert.assertEquals("Should have 1 record", 1, resultLTE.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultLTE.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIn() {
+    String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME);
+    String explainIN = getTableEnv().explainSql(sqlIN);
+    String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)";
+    Assert.assertTrue("Explain should contain the push down filter", explainIN.contains(expectedFilter));
+    List<Object[]> resultIN = sql(sqlIN);
+    Assert.assertEquals("Should have 2 records", 2, resultIN.size());
+
+    List<Object[]> expectedIN = Lists.newArrayList();
+    expectedIN.add(new Object[] {1, "iceberg", 10.0});
+    expectedIN.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedIN.toArray(), resultIN.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownInNull() {
+    String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME);
+    String explainInNull = getTableEnv().explainSql(sqlInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainInNull.contains(expectedFilterPushDownExplain));
+
+    List<Object[]> result = sql(sqlInNull);
+    Assert.assertEquals("Should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, result.get(0));
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownNotIn() {
+    String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME);
+    String explainNotIn = getTableEnv().explainSql(sqlNotIn);
+    String expectedFilter = "ref(name=\"id\") != 3,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotIn.contains(expectedFilter));
+
+    List<Object[]> resultNotIn = sql(sqlNotIn);
+    Assert.assertEquals("Should have 1 record", 1, resultNotIn.size());
+    Assert
+        .assertArrayEquals("Should produce the expected record", new Object[] {1, "iceberg", 10.0}, resultNotIn.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expectedScan = "(ref(name=\"id\") != 3 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedScan, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotInNull() {
+    String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME);
+    String explainNotInNull = getTableEnv().explainSql(sqlNotInNull);
+    Assert.assertFalse("Explain should not contain FilterPushDown",
+        explainNotInNull.contains(expectedFilterPushDownExplain));
+    List<Object[]> resultGT = sql(sqlNotInNull);
+    Assert.assertEquals("Should have 0 record", 0, resultGT.size());
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+  }
+
+  @Test
+  public void testFilterPushDownIsNotNull() {
+    String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME);
+    String explainNotNull = getTableEnv().explainSql(sqlNotNull);
+    String expectedFilter = "not_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotNull.contains(expectedFilter));
+
+    List<Object[]> resultNotNull = sql(sqlNotNull);
+    Assert.assertEquals("Should have 2 record", 2, resultNotNull.size());
+
+    List<Object[]> expected = Lists.newArrayList();
+    expected.add(new Object[] {1, "iceberg", 10.0});
+    expected.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expected.toArray(), resultNotNull.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownIsNull() {
+    String sqlNull = String.format("SELECT * FROM %s WHERE data IS  NULL", TABLE_NAME);
+    String explainNull = getTableEnv().explainSql(sqlNull);
+    String expectedFilter = "is_null(ref(name=\"data\"))";
+    Assert.assertTrue("Explain should contain the push down filter", explainNull.contains(expectedFilter));
+
+    List<Object[]> resultNull = sql(sqlNull);
+    Assert.assertEquals("Should have 1 record", 1, resultNull.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNull.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNot() {
+    String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME);
+    String explainNot = getTableEnv().explainSql(sqlNot);
+    String expectedFilter = "ref(name=\"id\") != 1,ref(name=\"id\") != 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainNot.contains(expectedFilter));
+
+    List<Object[]> resultNot = sql(sqlNot);
+    Assert.assertEquals("Should have 1 record", 1, resultNot.size());
+    Assert.assertArrayEquals("Should produce the expected record", new Object[] {3, null, 30.0}, resultNot.get(0));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)";
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownBetween() {
+    String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME);
+    String explainBetween = getTableEnv().explainSql(sqlBetween);
+    String expectedFilter = "ref(name=\"id\") >= 1,ref(name=\"id\") <= 2";
+    Assert.assertTrue("Explain should contain the push down filter", explainBetween.contains(expectedFilter));
+
+    List<Object[]> resultBetween = sql(sqlBetween);
+    Assert.assertEquals("Should have 2 record", 2, resultBetween.size());
+
+    List<Object[]> expectedBetween = Lists.newArrayList();
+    expectedBetween.add(new Object[] {1, "iceberg", 10.0});
+    expectedBetween.add(new Object[] {2, "b", 20.0});
+    Assert.assertArrayEquals("Should produce the expected record", expectedBetween.toArray(), resultBetween.toArray());
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)";
+    Assert.assertEquals("Should contain the push down filter", expected, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownNotBetween() {
+    String sqlNotBetween = String.format("SELECT * FROM %s WHERE id  NOT BETWEEN 2 AND 3 ", TABLE_NAME);
+    String explainNotBetween = getTableEnv().explainSql(sqlNotBetween);
+    String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)";
+    Assert.assertTrue("Explain should contain the push down filter", explainNotBetween.contains(expectedFilter));
+
+    List<Object[]> resultNotBetween = sql(sqlNotBetween);
+    Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size());
+    Assert.assertArrayEquals("the not between Should produce the expected record", resultNotBetween.get(0),
+        new Object[] {1, "iceberg", 10.0});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterPushDownLike() {
+    String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%' ";
+    String explainLike = getTableEnv().explainSql(sqlLike);
+    String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\"";
+    Assert.assertTrue("the like sql Explain should contain the push down filter", explainLike.contains(expectedFilter));
+
+    sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' ";
+    List<Object[]> resultLike = sql(sqlLike);
+    Assert.assertEquals("Should have 1 record", 1, resultLike.size());
+    Assert.assertArrayEquals("The like result should produce the expected record",
+        new Object[] {1, "iceberg", 10.0}, resultLike.get(0));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
+  }
+
+  @Test
+  public void testFilterNotPushDownLike() {

Review comment:
       I add the test case 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553163519



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +95,396 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
     Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));
     List<Object[]> result = sql(querySql);
-    Assert.assertEquals("should have 1 record", 1, result.size());
-    Assert.assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, result.size());
+    assertArrayEquals("Should produce the expected records", result.get(0), new Object[] {1, "a"});
 
     AssertHelpers.assertThrows("Invalid limit number: -1 ", SqlParserException.class,
         () -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME));
 
-    Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
+    assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
-    Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
+    expectedList.add(new Object[] {3, null});
+    assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
-    Assert.assertEquals("should have 1 record", 1, mixedResult.size());
-    Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+    assertEquals("should have 1 record", 1, mixedResult.size());
+    assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
+  }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));

Review comment:
       Asserting that the explain contains `FilterPushDown` does not have much meaning because all explained string will contains that word ( see the `IcebergTableSource#explainSource` ).   The key part to assert is the last part: 
   
   ```java
   String.format(", FilterPushDown,the filters :%s", Joiner.on(",").join(filters));
   ```
   
   I mean we need to assert  `Joiner.on(",").join(filters)` part ( for all the following cases).




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r556985794



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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.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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  /**
+   * convert flink expression to iceberg expression.
+   * <p>
+   * The BETWEEN, NOT_BETWEEN,IN expression will be converted by flink automatically. the BETWEEN will be converted to
+   * (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR GT_EQ), the IN will be converted to OR, so we do
+   * not add the conversion here
+   *
+   * @param flinkExpression the flink expression
+   * @return the iceberg expression
+   */
+  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:
+          Optional<String> name = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(singleton(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, Expressions::greaterThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, Expressions::lessThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call);
+
+        case EQ:
+          return handleNaN(Expressions::equal, Expressions::isNaN, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, Expressions::notNaN, call);
+
+        case NOT:
+          Optional<Expression> child = convert(singleton(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> singleton(CallExpression call,
+                                                                      Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = parseFieldAndLiteral(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    return Optional.empty();
+  }
+
+  private static Optional<Expression> convertLogicExpression(BiFunction<Expression, Expression, Expression> function,
+                                                             CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();

Review comment:
       If this is going to access children from this list directly using `get(1)` then it should also check that there are exactly 2 children.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-756933941


   I'll take another look.


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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557802667



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -73,15 +73,15 @@ protected TableEnvironment getTableEnv() {
   }
 
   protected static TableResult exec(TableEnvironment env, String query, Object... args) {
-    return env.executeSql(String.format(query, args));
+    return env.executeSql(args.length > 0 ? String.format(query, args) : query);

Review comment:
       if the sql contains `%` , for example `LIKE  'a%'`, the `String.format` method will throws an excpetion, so I do the change.




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r553161188



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.lit;
+import static org.junit.Assert.assertEquals;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(10))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_TIME_ZONE())
+      .field("field13", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, new byte[] {'a', 'b'});
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));
+
+    LocalDate date = LocalDate.parse("2020-12-23");
+    matchLiteral("field9", date, DateTimeUtil.daysFromDate(date));
+
+    LocalTime time = LocalTime.parse("12:13:14");
+    matchLiteral("field10", time, DateTimeUtil.microsFromTime(time));
+
+    LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14");
+    matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime));
+
+    Instant instant = Instant.parse("2020-12-23T12:13:14.00Z");
+    matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant));
+
+    matchLiteral("field13", instant, DateTimeUtil.microsFromInstant(instant));
+  }
+
+  @Test
+  public void testEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.equal("field1", 1);
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field1").isEqual(Expressions.lit(1)))).orElse(null);
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(1).isEqual(Expressions.$("field1")))).orElse(null);
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    UnboundPredicate<Float> expected = org.apache.iceberg.expressions.Expressions.isNaN("field3");
+
+    org.apache.iceberg.expressions.Expression actual =
+        FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN)))).orElse(null);
+    assertPredicatesMatch(expected, (UnboundPredicate) actual);
+
+    org.apache.iceberg.expressions.Expression actual1 =
+        FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3")))).orElse(null);
+    assertPredicatesMatch(expected, (UnboundPredicate) actual1);
+  }
+
+  @Test
+  public void testNotEquals() {
+    UnboundPredicate<Integer> expected = org.apache.iceberg.expressions.Expressions.notEqual("field1", 1);

Review comment:
       It's good to have test cases for all other data types except `Integer`.  I guess we may need few abstraction so that the test code won't duplicate too many. 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551218250



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  public static Optional<Expression> convert(org.apache.flink.table.expressions.Expression flinkExpression) {
+    if (flinkExpression == null || !(flinkExpression instanceof CallExpression)) {
+      return Optional.empty();
+    }
+
+    CallExpression call = (CallExpression) flinkExpression;
+    Operation op = FILTERS.get(call.getFunctionDefinition());
+    if (op != null) {
+      switch (op) {
+        case IS_NULL:
+          Optional<String> name = toReference(getOnlyChild(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(getOnlyChild(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        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 handleNaN(Expressions::equal, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, call);
+
+        case IN:
+          List<ResolvedExpression> args = call.getResolvedChildren();
+          Optional<String> fieldName = toReference(args.get(0));
+          List<ResolvedExpression> values = args.subList(1, args.size());
+
+          List<Object> inputValues = values.stream().filter(expression -> {
+            if (expression instanceof ValueLiteralExpression) {
+              return !((ValueLiteralExpression) expression).isNull();
+            }
+
+            return false;
+          }).map(expression -> {
+            Optional<Object> value = toLiteral(expression);
+            return value.get();
+          }).collect(Collectors.toList());
+
+          return Optional.of(Expressions.in(fieldName.get(), inputValues));
+
+        case NOT:
+          Optional<Expression> child = convert(getOnlyChild(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> getOnlyChild(CallExpression call,
+                                                                         Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 != null) {
+      return Optional.of(function.apply(tuple2.f0, tuple2.f1));

Review comment:
       Pls consider the two cases: 
   
   ```java
   Case.1 :   a < 1; 
   Case.2:    1 < a;
   ```
   
   Here we  `convertBinaryExpress` will parse the `tuple2` as `<a, 1>`.  and the `function` will be  `lessThan`.  While in the actual the case.2 is totally different with case.1 because its meaning is :  a is `greaterThan` 1 . 
   
   That's why we introduced a reversed function in [here](https://github.com/generic-datalake/iceberg-poc/pull/2/files#diff-86160616589acf1dd526b10b73418a46fe60f9e5e5ab6946a4ea3c8f019542f5R65),  for example,  the `greaterThan`'s revered function is `lessThan`,  the `lessThan`'s reversed function is `greaterThan`.   If the case `1 < a` then we should call the reversed function here. 
   
   




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551209205



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

Review comment:
       OK,  make sense to me ! 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547979361



##########
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);

Review comment:
       Pls check its class type before casting to `FieldReferenceExpression` directly.  It's more safe to cast if we're sure that it's indeed the expected class. 




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557562832



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -90,17 +103,418 @@ public void testLimitPushDown() {
 
     Assert.assertEquals("should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size());
 
-    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 3", TABLE_NAME);
+    String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME);
     List<Object[]> resultExceed = sql(sqlLimitExceed);
-    Assert.assertEquals("should have 2 record", 2, resultExceed.size());
+    Assert.assertEquals("should have 3 record", 3, resultExceed.size());
     List<Object[]> expectedList = Lists.newArrayList();
     expectedList.add(new Object[] {1, "a"});
     expectedList.add(new Object[] {2, "b"});
+    expectedList.add(new Object[] {3, null});
     Assert.assertArrayEquals("Should produce the expected records", resultExceed.toArray(), expectedList.toArray());
 
     String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
     List<Object[]> mixedResult = sql(sqlMixed);
     Assert.assertEquals("should have 1 record", 1, mixedResult.size());
     Assert.assertArrayEquals("Should produce the expected records", mixedResult.get(0), new Object[] {1, "a"});
   }
+
+  @Test
+  public void testNoFilterPushDown() {
+    String sql = String.format("SELECT * FROM %s ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sql);
+    Assert.assertFalse("explain should no contains FilterPushDown", explain.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqual() {
+    String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME);
+    String explain = getTableEnv().explainSql(sqlLiteralRight);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explain.contains(expectedFilter));
+
+    List<Object[]> result = sql(sqlLiteralRight);
+    Assert.assertEquals("should have 1 record", 1, result.size());
+    Assert.assertArrayEquals("Should produce the expected record", result.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    // filter not push down
+    String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME);
+    String explainEqualNull = getTableEnv().explainSql(sqlEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown",
+        explainEqualNull.contains(expectedFilterPushDownExplain));
+  }
+
+  @Test
+  public void testFilterPushDownEqualLiteralOnLeft() {
+    String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME);
+    String explainLeft = getTableEnv().explainSql(sqlLiteralLeft);
+    String expectedFilter = "ref(name=\"id\") == 1";
+    Assert.assertTrue("explain should contains the push down filter", explainLeft.contains(expectedFilter));
+
+    List<Object[]> resultLeft = sql(sqlLiteralLeft);
+    Assert.assertEquals("should have 1 record", 1, resultLeft.size());
+    Assert.assertArrayEquals("Should produce the expected record", resultLeft.get(0), new Object[] {1, "a"});
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+  }
+
+  @Test
+  public void testFilterPushDownNoEqual() {
+    String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME);
+    String explainNE = getTableEnv().explainSql(sqlNE);
+    String expectedFilter = "ref(name=\"id\") != 1";
+    Assert.assertTrue("explain should contains the push down filter", explainNE.contains(expectedFilter));
+
+    List<Object[]> resultNE = sql(sqlNE);
+    Assert.assertEquals("should have 2 record", 2, resultNE.size());
+
+    List<Object[]> expectedNE = Lists.newArrayList();
+    expectedNE.add(new Object[] {2, "b"});
+    expectedNE.add(new Object[] {3, null});
+    Assert.assertArrayEquals("Should produce the expected record", resultNE.toArray(), expectedNE.toArray());
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("should contains the push down filter", lastScanEvent.filter().toString(), expectedFilter);
+
+    String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME);
+    String explainNotEqualNull = getTableEnv().explainSql(sqlNotEqualNull);
+    Assert.assertFalse("explain should not contains FilterPushDown", explainNotEqualNull.contains(
+        expectedFilterPushDownExplain));

Review comment:
       This should also check the scan event.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557016058



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSource.java
##########
@@ -75,32 +95,436 @@ public void clean() {
 
   @Test
   public void testLimitPushDown() {
-    sql("INSERT INTO %s  VALUES (1,'a'),(2,'b')", TABLE_NAME);
-
     String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME);
     String explain = getTableEnv().explainSql(querySql);
     String expectedExplain = "LimitPushDown : 1";
-    Assert.assertTrue("explain should contains LimitPushDown", explain.contains(expectedExplain));

Review comment:
       I don't think this line needed to change.




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557543457



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -73,15 +73,15 @@ protected TableEnvironment getTableEnv() {
   }
 
   protected static TableResult exec(TableEnvironment env, String query, Object... args) {
-    return env.executeSql(String.format(query, args));
+    return env.executeSql(args.length > 0 ? String.format(query, args) : query);
   }
 
   protected TableResult exec(String query, Object... args) {
     return exec(getTableEnv(), query, args);
   }
 
   protected List<Object[]> sql(String query, Object... args) {
-    TableResult tableResult = exec(String.format(query, args));
+    TableResult tableResult = exec(query, args);

Review comment:
       I think that this change is correct, but I don't see why it was necessary in this commit. Wouldn't all of the tests run the same way without modifying this file at all?




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


[GitHub] [iceberg] rdblue edited a comment on pull request #1893: Flink: Support filter pushdown in IcebergTableSource

Posted by GitBox <gi...@apache.org>.
rdblue edited a comment on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-761613165


   @zhangjun0x01, there are still a few things to fix in the tests, mostly minor. But I also found a major problem, which is that `TestFlinkTableSource` now takes a very long time to run. The problem is that the tests run for each format and for 3 different catalog configurations. That means each test runs 9 times and because it is a test that actually runs SQL it takes a long time. The whole suite takes much longer than needed; on my machine, it takes about 4 minutes.
   
   The filter pushdown tests only need to run for one catalog and one file format because the purpose of those tests is to validate the assumptions of the `FlinkFilter` class with real filters from Flink SQL. The file format and catalog are orthogonal and we don't need to test each one of them. Can you change the parameterization to run with only Avro and a single catalog case?


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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547977619



##########
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()));

Review comment:
       Yes , flink does not support nested field push down now.  Will need to file issue to address it in apache flink repo.




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546480034



##########
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()));

Review comment:
       I tested it, for example, we have a table,the flink ddl is like this:
   ```sql
   CREATE TABLE iceberg_nested_test (
                id VARCHAR,
                title VARCHAR,
                properties ROW(`foo` VARCHAR)
   ) WITH (
        'connector'='iceberg'
   );
   
   ```
   if the query sql is `select * from iceberg_nested_test where properties is null` ,it supports filter push down in flink, and the name is `properties`,if the sql is `select * from iceberg_nested_test where properties.foo is null`,it do not supports filter push down in flink




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-749547461


   I'm sorry that I did not review this PR in time before (was focusing on flink cdc DataStream/SQL test cases and more optimizations things after the next release 0.11.0), will take a look tomorrow. 


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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557804089



##########
File path: flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java
##########
@@ -73,15 +73,15 @@ protected TableEnvironment getTableEnv() {
   }
 
   protected static TableResult exec(TableEnvironment env, String query, Object... args) {
-    return env.executeSql(String.format(query, args));
+    return env.executeSql(args.length > 0 ? String.format(query, args) : query);
   }
 
   protected TableResult exec(String query, Object... args) {
     return exec(getTableEnv(), query, args);
   }
 
   protected List<Object[]> sql(String query, Object... args) {
-    TableResult tableResult = exec(String.format(query, args));
+    TableResult tableResult = exec(query, args);

Review comment:
       the `LIKE` sql in test case will call this method, I think it is maybe a small mistake, so I update it   to pass the original parameters `query, args`, in the final method `exec(TableEnvironment env, String query, Object... args)` to do the format




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


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

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r557554809



##########
File path: flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.expressions.ApiExpressionUtils;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.And;
+import org.apache.iceberg.expressions.BoundLiteralPredicate;
+import org.apache.iceberg.expressions.Not;
+import org.apache.iceberg.expressions.Or;
+import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFlinkFilters {
+
+  private static final TableSchema TABLE_SCHEMA = TableSchema.builder()
+      .field("field1", DataTypes.INT())
+      .field("field2", DataTypes.BIGINT())
+      .field("field3", DataTypes.FLOAT())
+      .field("field4", DataTypes.DOUBLE())
+      .field("field5", DataTypes.STRING())
+      .field("field6", DataTypes.BOOLEAN())
+      .field("field7", DataTypes.BINARY(2))
+      .field("field8", DataTypes.DECIMAL(10, 2))
+      .field("field9", DataTypes.DATE())
+      .field("field10", DataTypes.TIME())
+      .field("field11", DataTypes.TIMESTAMP())
+      .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())
+      .build();
+
+  // A map list of fields and values used to verify the conversion of flink expression to iceberg expression
+  private static final List<Tuple2<String, Object>> FIELD_VALUE_LIST = ImmutableList.of(
+      Tuple2.of("field1", 1),
+      Tuple2.of("field2", 2L),
+      Tuple2.of("field3", 3F),
+      Tuple2.of("field4", 4D),
+      Tuple2.of("field5", "iceberg"),
+      Tuple2.of("field6", true),
+      Tuple2.of("field7", new byte[] {'a', 'b'}),
+      Tuple2.of("field8", BigDecimal.valueOf(10)),
+      Tuple2.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())),
+      Tuple2.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())),
+      Tuple2.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())),
+      Tuple2.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))
+  );
+
+  @Test
+  public void testFlinkDataTypeEqual() {
+    matchLiteral("field1", 1, 1);
+    matchLiteral("field2", 10L, 10L);
+    matchLiteral("field3", 1.2F, 1.2F);
+    matchLiteral("field4", 3.4D, 3.4D);
+    matchLiteral("field5", "abcd", "abcd");
+    matchLiteral("field6", true, true);
+    matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'}));
+    matchLiteral("field8", BigDecimal.valueOf(10), BigDecimal.valueOf(10));

Review comment:
       `valueOf` creates a decimal with a scale of 0, but the type's scale is 2. Can you create a big decimal that matches the type?




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


[GitHub] [iceberg] rdblue edited a comment on pull request #1893: Flink: Support filter pushdown in IcebergTableSource

Posted by GitBox <gi...@apache.org>.
rdblue edited a comment on pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#issuecomment-761613165


   @zhangjun0x01, there are still a few things to fix in the tests, mostly minor. But I also found a major problem, which is that `TestFlinkTableSource` now takes a very long time to run. The problem is that the tests run for each format and for 3 different catalog configurations. That means each test runs 9 times and because it is a test that actually runs SQL it takes a long time. The whole suite takes much longer than needed; on my machine, it takes more than 10 minutes!
   
   The filter pushdown tests only need to run for one catalog and one file format because the purpose of those tests is to validate the assumptions of the `FlinkFilter` class with real filters from Flink SQL. The file format and catalog are orthogonal and we don't need to test each one of them. Can you change the parameterization to run with only Avro and a single catalog case?


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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551696789



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.flink.api.java.tuple.Tuple2;
+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.BuiltInFunctionDefinitions;
+import org.apache.flink.table.functions.FunctionDefinition;
+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;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.NaNUtil;
+
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%");
+
+  private static final Map<FunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<FunctionDefinition, 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)
+      .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH)
+      .build();
+
+  public static Optional<Expression> convert(org.apache.flink.table.expressions.Expression flinkExpression) {
+    if (flinkExpression == null || !(flinkExpression instanceof CallExpression)) {
+      return Optional.empty();
+    }
+
+    CallExpression call = (CallExpression) flinkExpression;
+    Operation op = FILTERS.get(call.getFunctionDefinition());
+    if (op != null) {
+      switch (op) {
+        case IS_NULL:
+          Optional<String> name = toReference(getOnlyChild(call, FieldReferenceExpression.class).orElse(null));
+          return name.map(Expressions::isNull);
+
+        case NOT_NULL:
+          Optional<String> nameNotNull = toReference(getOnlyChild(call, FieldReferenceExpression.class).orElse(null));
+          return nameNotNull.map(Expressions::notNull);
+
+        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 handleNaN(Expressions::equal, call);
+
+        case NOT_EQ:
+          return handleNaN(Expressions::notEqual, call);
+
+        case IN:
+          List<ResolvedExpression> args = call.getResolvedChildren();
+          Optional<String> fieldName = toReference(args.get(0));
+          List<ResolvedExpression> values = args.subList(1, args.size());
+
+          List<Object> inputValues = values.stream().filter(expression -> {
+            if (expression instanceof ValueLiteralExpression) {
+              return !((ValueLiteralExpression) expression).isNull();
+            }
+
+            return false;
+          }).map(expression -> {
+            Optional<Object> value = toLiteral(expression);
+            return value.get();
+          }).collect(Collectors.toList());
+
+          return Optional.of(Expressions.in(fieldName.get(), inputValues));
+
+        case NOT:
+          Optional<Expression> child = convert(getOnlyChild(call, CallExpression.class).orElse(null));
+          return child.map(Expressions::not);
+
+        case AND:
+          return convertLogicExpression(Expressions::and, call);
+
+        case OR:
+          return convertLogicExpression(Expressions::or, call);
+
+        case STARTS_WITH:
+          return convertLike(call);
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static <T extends ResolvedExpression> Optional<T> getOnlyChild(CallExpression call,
+                                                                         Class<T> expectedChildClass) {
+    List<ResolvedExpression> children = call.getResolvedChildren();
+    if (children.size() != 1) {
+      return Optional.empty();
+    }
+
+    ResolvedExpression child = children.get(0);
+    if (!expectedChildClass.isInstance(child)) {
+      return Optional.empty();
+    }
+
+    return Optional.of(expectedChildClass.cast(child));
+  }
+
+  private static Optional<Expression> convertLike(CallExpression call) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 == null) {
+      return Optional.empty();
+    }
+
+    String pattern = tuple2.f1.toString();
+    Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern);
+
+    // exclude special char of LIKE
+    // '_' is the wildcard of the SQL LIKE
+    if (!pattern.contains("_") && matcher.matches()) {
+      return Optional.of(Expressions.startsWith(tuple2.f0, matcher.group(1)));
+    }
+
+    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) {
+    Tuple2<String, Object> tuple2 = convertBinaryExpress(call);
+    if (tuple2 != null) {
+      return Optional.of(function.apply(tuple2.f0, tuple2.f1));

Review comment:
       I add the reversedFunction in `lessThan`,`lessThanOrEqual`,`greaterThan`,`greaterThanOrEqual`.
   and add the test case 




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


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

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r551980613



##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java
##########
@@ -33,33 +37,37 @@
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.utils.TableConnectorUtils;
 import org.apache.iceberg.flink.source.FlinkSource;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 
 /**
  * Flink Iceberg table source.
- * TODO: Implement {@link FilterableTableSource}
  */
 public class IcebergTableSource
-    implements StreamTableSource<RowData>, ProjectableTableSource<RowData>, LimitableTableSource<RowData> {
+    implements StreamTableSource<RowData>, ProjectableTableSource<RowData>, FilterableTableSource<RowData>,
+    LimitableTableSource<RowData> {
 
   private final TableLoader loader;
   private final TableSchema schema;
   private final Map<String, String> properties;
   private final int[] projectedFields;
   private final boolean isLimitPushDown;
   private final long limit;
+  private final List<org.apache.iceberg.expressions.Expression> filters;
 
   public IcebergTableSource(TableLoader loader, TableSchema schema, Map<String, String> properties) {
-    this(loader, schema, properties, null, false, -1);
+    this(loader, schema, properties, null, false, -1, null);

Review comment:
       It's better to use `ImmutableList.of()`  as the default `filters` because of the comment [here](https://github.com/apache/iceberg/pull/1893/files#r544779804), though there's large probability that when `isFilterPushedDown()` returns true the filters should always be non-nullable. 




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


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

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r546480034



##########
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()));

Review comment:
       I tested it, for example, we have a table,the flink ddl is like this:
   ```sql
   CREATE TABLE iceberg_nested_test (
                id VARCHAR,
                title VARCHAR,
                properties ROW(`foo` VARCHAR)
   ) WITH (
        'connector'='iceberg'
   );
   
   ```
   if the query sql is `select * from iceberg_nested_test where properties is null` ,it supports filter push down in flink, and the name is `properties`,if the sql is `select * from iceberg_nested_test where properties.foo is null`,it do not supports filter push down in flink,the code will do not enter the `IcebergTableSource#applyPredicate` method 




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