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/02 00:36:35 UTC

[GitHub] [iceberg] yyanyy opened a new pull request #1857: rewrite equals filters

yyanyy opened a new pull request #1857:
URL: https://github.com/apache/iceberg/pull/1857


   This PR is based on #1747. Currently it couldn't compile but I have confirmed locally that the project could build and all tests passed after rebasing it on top of #1747. This is because there are a few files required from #1747 to let the tests work. I will rebase the change and mark as ready for review once 1747 is merged. 


----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java
##########
@@ -63,11 +63,25 @@ public void removeTables() {
 
   @Test
   public void testSelect() {
-    List<Object[]> expected = ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"));
+    List<Object[]> expected = ImmutableList.of(
+        row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN));
 
     assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName));
   }
 
+  @Test
+  public void testSelectRewrite() {
+    List<Object[]> expected = ImmutableList.of(row(3L, "c", Float.NaN));
+
+    assertEquals("Should return all expected rows", expected,
+        sql("SELECT * FROM %s where float = float('NaN')", tableName));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter",
+        "(float IS NOT NULL AND float = NaN)",

Review comment:
       Yes, I think so. The description shouldn't produce a predicate that we don't support!




----------------------------------------------------------------
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 #1857: rewrite equals filters

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


   Thanks, @yyanyy! Looks good.


----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark2/src/test/java/org/apache/iceberg/spark/source/TestSelect.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.spark.source;
+
+import java.io.File;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.events.ScanEvent;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class TestSelect {
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()),
+      optional(3, "doubleVal", Types.DoubleType.get())
+  );
+
+  private static SparkSession spark;
+
+  private static int scanEventCount = 0;
+  private static ScanEvent lastScanEvent = null;
+
+  private Table table;
+
+  static {
+    Listeners.register(event -> {
+      scanEventCount += 1;
+      lastScanEvent = event;
+    }, ScanEvent.class);
+  }
+
+  @BeforeClass
+  public static void startSpark() {
+    spark = SparkSession.builder()
+        .master("local[2]")
+        .getOrCreate();
+  }
+
+  @AfterClass
+  public static void stopSpark() {
+    SparkSession currentSpark = spark;
+    spark = null;
+    currentSpark.stop();
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void init() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+
+    table = TABLES.create(SCHEMA, tableLocation);
+
+    List<Record> rows = Lists.newArrayList(
+        new Record(1, "a", 1.0),
+        new Record(2, "b", 2.0),
+        new Record(3, "c", Double.NaN)
+    );
+
+    Dataset<Row> df = spark.createDataFrame(rows, Record.class);
+
+    df.select("id", "data", "doubleVal").write()
+        .format("iceberg")
+        .mode("append")
+        .save(tableLocation);
+
+    table.refresh();
+
+    Dataset<Row> results = spark.read()
+        .format("iceberg")
+        .load(tableLocation);
+    results.createOrReplaceTempView("table");
+
+    scanEventCount = 0;
+    lastScanEvent = null;
+  }
+
+  @Test
+  public void testSelect() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, "a", 1.0), new Record(2, "b", 2.0), new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("select * from table"));
+  }
+
+  @Test
+  public void testSelectRewrite() {
+    List<Record> expected = ImmutableList.of(new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected,
+        sql("SELECT * FROM table where doubleVal = double('NaN')"));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+  }
+
+  @Test
+  public void testProjection() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, null, null), new Record(2, null, null), new Record(3, null, null));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("SELECT id FROM table"));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+    Assert.assertEquals("Should project only the id column",
+        table.schema().select("id").asStruct(),
+        lastScanEvent.projection().asStruct());
+  }
+
+  @Test
+  public void testExpressionPushdown() {
+    List<Record> expected = ImmutableList.of(new Record(null, "b", null));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("SELECT data FROM table WHERE id = 2"));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should project only id and data columns",
+        table.schema().select("id", "data").asStruct(),
+        lastScanEvent.projection().asStruct());
+  }
+
+  private List<Record> sql(String str) {
+    List<Row> rows = spark.sql(str).collectAsList();

Review comment:
       Sounds good, I wanted to scope this `Record` class to only be used for this class' use cases but this is definitely not clean. I changed this to use Spark for converting, but encountered a similar issue as described in [this post](https://stackoverflow.com/questions/44886211/spark-default-null-columns-dataset) that when projecting a subset of columns, conversion doesn't work. Since in this class I'm just projecting one column with primitive type, I convert data frame into their specific classes instead. Please let me know if you know better ways of doing 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] rdblue commented on pull request #1857: rewrite equals filters

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


   I merged #1747, so you can rebase this. 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] rdblue merged pull request #1857: rewrite equals filters

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


   


----------------------------------------------------------------
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] jackye1995 commented on a change in pull request #1857: rewrite equals filters

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



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/SparkFilters.java
##########
@@ -113,13 +115,13 @@ public static Expression convert(Filter filter) {
             // comparison with null in normal equality is always null. this is probably a mistake.
             Preconditions.checkNotNull(eq.value(),
                 "Expression is always false (eq is not null-safe): %s", filter);
-            return equal(eq.attribute(), convertLiteral(eq.value()));
+            return handleEqual(eq.attribute(), eq.value());
           } else {
             EqualNullSafe eq = (EqualNullSafe) filter;
             if (eq.value() == null) {
               return isNull(eq.attribute());
             } else {
-              return equal(eq.attribute(), convertLiteral(eq.value()));
+              return handleEqual(eq.attribute(), eq.value());

Review comment:
       why is this not directly inside `Expressions.equal`, so we can avoid duplication between spark 2 and 3?




----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java
##########
@@ -49,8 +49,8 @@ public TestSelect(String catalogName, String implementation, Map<String, String>
 

Review comment:
       Added a `TestSelect` in spark2 by basically duplicating logic from the same class in spark3; although apart from basic sanity testing I'm not sure how helpful the tests are as some of the logic for examine pushed-down filters only exist in spark3...




----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java
##########
@@ -63,11 +63,25 @@ public void removeTables() {
 
   @Test
   public void testSelect() {
-    List<Object[]> expected = ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"));
+    List<Object[]> expected = ImmutableList.of(
+        row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN));
 
     assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName));
   }
 
+  @Test
+  public void testSelectRewrite() {
+    List<Object[]> expected = ImmutableList.of(row(3L, "c", Float.NaN));
+
+    assertEquals("Should return all expected rows", expected,
+        sql("SELECT * FROM %s where float = float('NaN')", tableName));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter",
+        "(float IS NOT NULL AND float = NaN)",

Review comment:
       This is because in `DescribeExpressionVisitor ` we translate is_nan to `= NaN` in [here](https://github.com/apache/iceberg/blob/master/spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java#L536). Do you want me to change this to `is_nan(float)`? 

##########
File path: spark2/src/test/java/org/apache/iceberg/spark/source/TestSelect.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.spark.source;
+
+import java.io.File;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.events.ScanEvent;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class TestSelect {
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()),
+      optional(3, "doubleVal", Types.DoubleType.get())
+  );
+
+  private static SparkSession spark;
+
+  private static int scanEventCount = 0;
+  private static ScanEvent lastScanEvent = null;
+
+  private Table table;
+
+  static {
+    Listeners.register(event -> {
+      scanEventCount += 1;
+      lastScanEvent = event;
+    }, ScanEvent.class);
+  }
+
+  @BeforeClass
+  public static void startSpark() {
+    spark = SparkSession.builder()
+        .master("local[2]")
+        .getOrCreate();
+  }
+
+  @AfterClass
+  public static void stopSpark() {
+    SparkSession currentSpark = spark;
+    spark = null;
+    currentSpark.stop();
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void init() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+
+    table = TABLES.create(SCHEMA, tableLocation);
+
+    List<Record> rows = Lists.newArrayList(
+        new Record(1, "a", 1.0),
+        new Record(2, "b", 2.0),
+        new Record(3, "c", Double.NaN)
+    );
+
+    Dataset<Row> df = spark.createDataFrame(rows, Record.class);
+
+    df.select("id", "data", "doubleVal").write()
+        .format("iceberg")
+        .mode("append")
+        .save(tableLocation);
+
+    table.refresh();
+
+    Dataset<Row> results = spark.read()
+        .format("iceberg")
+        .load(tableLocation);
+    results.createOrReplaceTempView("table");
+
+    scanEventCount = 0;
+    lastScanEvent = null;
+  }
+
+  @Test
+  public void testSelect() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, "a", 1.0), new Record(2, "b", 2.0), new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("select * from table"));
+  }
+
+  @Test
+  public void testSelectRewrite() {
+    List<Record> expected = ImmutableList.of(new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected,
+        sql("SELECT * FROM table where doubleVal = double('NaN')"));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);

Review comment:
       Yes, sorry I forgot to revisit this after cleaning up other changes. Since in spark2 we don't have `Spark3Util.describe()` I wasn't sure to which level we want to assert the expression, so that we can still have test coverage without being too coupled with internal implementation. Let me know how you think the updated test 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 #1857: rewrite equals filters

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



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java
##########
@@ -63,11 +63,25 @@ public void removeTables() {
 
   @Test
   public void testSelect() {
-    List<Object[]> expected = ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"));
+    List<Object[]> expected = ImmutableList.of(
+        row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN));
 
     assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName));
   }
 
+  @Test
+  public void testSelectRewrite() {
+    List<Object[]> expected = ImmutableList.of(row(3L, "c", Float.NaN));
+
+    assertEquals("Should return all expected rows", expected,
+        sql("SELECT * FROM %s where float = float('NaN')", tableName));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should push down expected filter",
+        "(float IS NOT NULL AND float = NaN)",

Review comment:
       Shouldn't this be `is_nan(float)` instead of `= NaN`?




----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark2/src/test/java/org/apache/iceberg/spark/source/TestSelect.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.spark.source;
+
+import java.io.File;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.events.ScanEvent;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class TestSelect {
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()),
+      optional(3, "doubleVal", Types.DoubleType.get())
+  );
+
+  private static SparkSession spark;
+
+  private static int scanEventCount = 0;
+  private static ScanEvent lastScanEvent = null;
+
+  private Table table;
+
+  static {
+    Listeners.register(event -> {
+      scanEventCount += 1;
+      lastScanEvent = event;
+    }, ScanEvent.class);
+  }
+
+  @BeforeClass
+  public static void startSpark() {
+    spark = SparkSession.builder()
+        .master("local[2]")
+        .getOrCreate();
+  }
+
+  @AfterClass
+  public static void stopSpark() {
+    SparkSession currentSpark = spark;
+    spark = null;
+    currentSpark.stop();
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void init() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+
+    table = TABLES.create(SCHEMA, tableLocation);
+
+    List<Record> rows = Lists.newArrayList(
+        new Record(1, "a", 1.0),
+        new Record(2, "b", 2.0),
+        new Record(3, "c", Double.NaN)
+    );
+
+    Dataset<Row> df = spark.createDataFrame(rows, Record.class);
+
+    df.select("id", "data", "doubleVal").write()
+        .format("iceberg")
+        .mode("append")
+        .save(tableLocation);
+
+    table.refresh();
+
+    Dataset<Row> results = spark.read()
+        .format("iceberg")
+        .load(tableLocation);
+    results.createOrReplaceTempView("table");
+
+    scanEventCount = 0;
+    lastScanEvent = null;
+  }
+
+  @Test
+  public void testSelect() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, "a", 1.0), new Record(2, "b", 2.0), new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("select * from table"));
+  }
+
+  @Test
+  public void testSelectRewrite() {
+    List<Record> expected = ImmutableList.of(new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected,
+        sql("SELECT * FROM table where doubleVal = double('NaN')"));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);

Review comment:
       Looks good!




----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark2/src/test/java/org/apache/iceberg/spark/source/TestSelect.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.spark.source;
+
+import java.io.File;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.events.ScanEvent;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class TestSelect {
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()),
+      optional(3, "doubleVal", Types.DoubleType.get())
+  );
+
+  private static SparkSession spark;
+
+  private static int scanEventCount = 0;
+  private static ScanEvent lastScanEvent = null;
+
+  private Table table;
+
+  static {
+    Listeners.register(event -> {
+      scanEventCount += 1;
+      lastScanEvent = event;
+    }, ScanEvent.class);
+  }
+
+  @BeforeClass
+  public static void startSpark() {
+    spark = SparkSession.builder()
+        .master("local[2]")
+        .getOrCreate();
+  }
+
+  @AfterClass
+  public static void stopSpark() {
+    SparkSession currentSpark = spark;
+    spark = null;
+    currentSpark.stop();
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void init() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+
+    table = TABLES.create(SCHEMA, tableLocation);
+
+    List<Record> rows = Lists.newArrayList(
+        new Record(1, "a", 1.0),
+        new Record(2, "b", 2.0),
+        new Record(3, "c", Double.NaN)
+    );
+
+    Dataset<Row> df = spark.createDataFrame(rows, Record.class);
+
+    df.select("id", "data", "doubleVal").write()
+        .format("iceberg")
+        .mode("append")
+        .save(tableLocation);
+
+    table.refresh();
+
+    Dataset<Row> results = spark.read()
+        .format("iceberg")
+        .load(tableLocation);
+    results.createOrReplaceTempView("table");
+
+    scanEventCount = 0;
+    lastScanEvent = null;
+  }
+
+  @Test
+  public void testSelect() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, "a", 1.0), new Record(2, "b", 2.0), new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("select * from table"));
+  }
+
+  @Test
+  public void testSelectRewrite() {
+    List<Record> expected = ImmutableList.of(new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected,
+        sql("SELECT * FROM table where doubleVal = double('NaN')"));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+  }
+
+  @Test
+  public void testProjection() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, null, null), new Record(2, null, null), new Record(3, null, null));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("SELECT id FROM table"));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+    Assert.assertEquals("Should project only the id column",
+        table.schema().select("id").asStruct(),
+        lastScanEvent.projection().asStruct());
+  }
+
+  @Test
+  public void testExpressionPushdown() {
+    List<Record> expected = ImmutableList.of(new Record(null, "b", null));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("SELECT data FROM table WHERE id = 2"));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should project only id and data columns",
+        table.schema().select("id", "data").asStruct(),
+        lastScanEvent.projection().asStruct());
+  }
+
+  private List<Record> sql(String str) {
+    List<Row> rows = spark.sql(str).collectAsList();

Review comment:
       This seems brittle because it uses types to place the results.
   
   Other tests use `StructProjection` and `StructLikeSet` for similar validations. The incoming row is wrapped to be a `StructLike` and added to a `StructLikeSet` based on the expected schema. Then another `StructLikeSet` is created with the expected rows, which are projected using `StructProjection` and the expected schema. That is a cleaner way to do this, I 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] yyanyy commented on a change in pull request #1857: rewrite equals filters

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



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/SparkFilters.java
##########
@@ -113,13 +115,13 @@ public static Expression convert(Filter filter) {
             // comparison with null in normal equality is always null. this is probably a mistake.
             Preconditions.checkNotNull(eq.value(),
                 "Expression is always false (eq is not null-safe): %s", filter);
-            return equal(eq.attribute(), convertLiteral(eq.value()));
+            return handleEqual(eq.attribute(), eq.value());
           } else {
             EqualNullSafe eq = (EqualNullSafe) filter;
             if (eq.value() == null) {
               return isNull(eq.attribute());
             } else {
-              return equal(eq.attribute(), convertLiteral(eq.value()));
+              return handleEqual(eq.attribute(), eq.value());

Review comment:
       I thought to reject NaN in any predicate and let `SparkFilters` to do rewrites was the conclusion we reached in [this thread](https://github.com/apache/iceberg/pull/1747#discussion_r529096291)?




----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark2/src/test/java/org/apache/iceberg/spark/source/TestSelect.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.spark.source;
+
+import java.io.File;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.events.ScanEvent;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class TestSelect {
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()),
+      optional(3, "doubleVal", Types.DoubleType.get())
+  );
+
+  private static SparkSession spark;
+
+  private static int scanEventCount = 0;
+  private static ScanEvent lastScanEvent = null;
+
+  private Table table;
+
+  static {
+    Listeners.register(event -> {
+      scanEventCount += 1;
+      lastScanEvent = event;
+    }, ScanEvent.class);
+  }
+
+  @BeforeClass
+  public static void startSpark() {
+    spark = SparkSession.builder()
+        .master("local[2]")
+        .getOrCreate();
+  }
+
+  @AfterClass
+  public static void stopSpark() {
+    SparkSession currentSpark = spark;
+    spark = null;
+    currentSpark.stop();
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void init() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+
+    table = TABLES.create(SCHEMA, tableLocation);
+
+    List<Record> rows = Lists.newArrayList(
+        new Record(1, "a", 1.0),
+        new Record(2, "b", 2.0),
+        new Record(3, "c", Double.NaN)
+    );
+
+    Dataset<Row> df = spark.createDataFrame(rows, Record.class);
+
+    df.select("id", "data", "doubleVal").write()
+        .format("iceberg")
+        .mode("append")
+        .save(tableLocation);
+
+    table.refresh();
+
+    Dataset<Row> results = spark.read()
+        .format("iceberg")
+        .load(tableLocation);
+    results.createOrReplaceTempView("table");
+
+    scanEventCount = 0;
+    lastScanEvent = null;
+  }
+
+  @Test
+  public void testSelect() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, "a", 1.0), new Record(2, "b", 2.0), new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("select * from table"));
+  }
+
+  @Test
+  public void testSelectRewrite() {
+    List<Record> expected = ImmutableList.of(new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected,
+        sql("SELECT * FROM table where doubleVal = double('NaN')"));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);

Review comment:
       Shouldn't this validate more than just the number of scans?




----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark2/src/test/java/org/apache/iceberg/spark/source/TestSelect.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.spark.source;
+
+import java.io.File;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.events.ScanEvent;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class TestSelect {
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()),
+      optional(3, "doubleVal", Types.DoubleType.get())
+  );
+
+  private static SparkSession spark;
+
+  private static int scanEventCount = 0;
+  private static ScanEvent lastScanEvent = null;
+
+  private Table table;
+
+  static {
+    Listeners.register(event -> {
+      scanEventCount += 1;
+      lastScanEvent = event;
+    }, ScanEvent.class);
+  }
+
+  @BeforeClass
+  public static void startSpark() {
+    spark = SparkSession.builder()
+        .master("local[2]")
+        .getOrCreate();
+  }
+
+  @AfterClass
+  public static void stopSpark() {
+    SparkSession currentSpark = spark;
+    spark = null;
+    currentSpark.stop();
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void init() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+
+    table = TABLES.create(SCHEMA, tableLocation);
+
+    List<Record> rows = Lists.newArrayList(
+        new Record(1, "a", 1.0),
+        new Record(2, "b", 2.0),
+        new Record(3, "c", Double.NaN)
+    );
+
+    Dataset<Row> df = spark.createDataFrame(rows, Record.class);
+
+    df.select("id", "data", "doubleVal").write()
+        .format("iceberg")
+        .mode("append")
+        .save(tableLocation);
+
+    table.refresh();
+
+    Dataset<Row> results = spark.read()
+        .format("iceberg")
+        .load(tableLocation);
+    results.createOrReplaceTempView("table");
+
+    scanEventCount = 0;
+    lastScanEvent = null;
+  }
+
+  @Test
+  public void testSelect() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, "a", 1.0), new Record(2, "b", 2.0), new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("select * from table"));
+  }
+
+  @Test
+  public void testSelectRewrite() {
+    List<Record> expected = ImmutableList.of(new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected,
+        sql("SELECT * FROM table where doubleVal = double('NaN')"));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+  }
+
+  @Test
+  public void testProjection() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, null, null), new Record(2, null, null), new Record(3, null, null));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("SELECT id FROM table"));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+    Assert.assertEquals("Should project only the id column",
+        table.schema().select("id").asStruct(),
+        lastScanEvent.projection().asStruct());
+  }
+
+  @Test
+  public void testExpressionPushdown() {
+    List<Record> expected = ImmutableList.of(new Record(null, "b", null));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("SELECT data FROM table WHERE id = 2"));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should project only id and data columns",
+        table.schema().select("id", "data").asStruct(),
+        lastScanEvent.projection().asStruct());
+  }
+
+  private List<Record> sql(String str) {
+    List<Row> rows = spark.sql(str).collectAsList();

Review comment:
       Looks like this uses a Java bean record class, so you could also rely on Spark to convert to your record class, and then use a special comparison function to only compare expected columns.




----------------------------------------------------------------
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] jackye1995 commented on a change in pull request #1857: rewrite equals filters

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



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java
##########
@@ -49,8 +49,8 @@ public TestSelect(String catalogName, String implementation, Map<String, String>
 

Review comment:
       I think we should also try to test it for spark2, maybe update some tests in `TestReadProjection`?




----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark2/src/test/java/org/apache/iceberg/spark/source/TestSelect.java
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.spark.source;
+
+import java.io.File;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.events.ScanEvent;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public class TestSelect {
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get()),
+      optional(3, "doubleVal", Types.DoubleType.get())
+  );
+
+  private static SparkSession spark;
+
+  private static int scanEventCount = 0;
+  private static ScanEvent lastScanEvent = null;
+
+  private Table table;
+
+  static {
+    Listeners.register(event -> {
+      scanEventCount += 1;
+      lastScanEvent = event;
+    }, ScanEvent.class);
+  }
+
+  @BeforeClass
+  public static void startSpark() {
+    spark = SparkSession.builder()
+        .master("local[2]")
+        .getOrCreate();
+  }
+
+  @AfterClass
+  public static void stopSpark() {
+    SparkSession currentSpark = spark;
+    spark = null;
+    currentSpark.stop();
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation = null;
+
+  @Before
+  public void init() throws Exception {
+    File tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+
+    table = TABLES.create(SCHEMA, tableLocation);
+
+    List<Record> rows = Lists.newArrayList(
+        new Record(1, "a", 1.0),
+        new Record(2, "b", 2.0),
+        new Record(3, "c", Double.NaN)
+    );
+
+    Dataset<Row> df = spark.createDataFrame(rows, Record.class);
+
+    df.select("id", "data", "doubleVal").write()
+        .format("iceberg")
+        .mode("append")
+        .save(tableLocation);
+
+    table.refresh();
+
+    Dataset<Row> results = spark.read()
+        .format("iceberg")
+        .load(tableLocation);
+    results.createOrReplaceTempView("table");
+
+    scanEventCount = 0;
+    lastScanEvent = null;
+  }
+
+  @Test
+  public void testSelect() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, "a", 1.0), new Record(2, "b", 2.0), new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("select * from table"));
+  }
+
+  @Test
+  public void testSelectRewrite() {
+    List<Record> expected = ImmutableList.of(new Record(3, "c", Double.NaN));
+
+    Assert.assertEquals("Should return all expected rows", expected,
+        sql("SELECT * FROM table where doubleVal = double('NaN')"));
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+  }
+
+  @Test
+  public void testProjection() {
+    List<Record> expected = ImmutableList.of(
+        new Record(1, null, null), new Record(2, null, null), new Record(3, null, null));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("SELECT id FROM table"));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
+    Assert.assertEquals("Should project only the id column",
+        table.schema().select("id").asStruct(),
+        lastScanEvent.projection().asStruct());
+  }
+
+  @Test
+  public void testExpressionPushdown() {
+    List<Record> expected = ImmutableList.of(new Record(null, "b", null));
+
+    Assert.assertEquals("Should return all expected rows", expected, sql("SELECT data FROM table WHERE id = 2"));
+
+    Assert.assertEquals("Should create only one scan", 1, scanEventCount);
+    Assert.assertEquals("Should project only id and data columns",
+        table.schema().select("id", "data").asStruct(),
+        lastScanEvent.projection().asStruct());
+  }
+
+  private List<Record> sql(String str) {
+    List<Row> rows = spark.sql(str).collectAsList();

Review comment:
       Sounds good, I wanted to scope this `Record` class to only be used for this class' use cases but this is definitely not clean. I changed this to use Spark for converting to Java bean, but encountered a similar issue as described in [this post](https://stackoverflow.com/questions/44886211/spark-default-null-columns-dataset) that when projecting a subset of columns, conversion doesn't work due to missing expected columns. Since in this class I'm just projecting one column with primitive type, I convert data frame into their specific classes instead. Please let me know if you know better ways of doing 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] yyanyy commented on a change in pull request #1857: rewrite equals filters

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



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java
##########
@@ -49,8 +49,8 @@ public TestSelect(String catalogName, String implementation, Map<String, String>
 

Review comment:
       Yeah I actually also spent some time on this but wasn't able to find a good place to add in spark2, and later gave up thinking that the added logic was relatively simple anyway. To me `TestReadProjection` is more about testing projection which is not what we are doing. I guess I'll create a `TestSelect` in spark3 test suite and duplicate this class then. 




----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/SparkFilters.java
##########
@@ -177,4 +179,13 @@ private static Object convertLiteral(Object value) {
     }
     return value;
   }
+
+  private static Expression handleEqual(String attribute, Object value) {
+    Object literal = convertLiteral(value);

Review comment:
       This should be moved into the `else` block because literal should not allow creating a `NaN` literal.




----------------------------------------------------------------
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 #1857: rewrite equals filters

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



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/SparkFilters.java
##########
@@ -113,13 +115,13 @@ public static Expression convert(Filter filter) {
             // comparison with null in normal equality is always null. this is probably a mistake.
             Preconditions.checkNotNull(eq.value(),
                 "Expression is always false (eq is not null-safe): %s", filter);
-            return equal(eq.attribute(), convertLiteral(eq.value()));
+            return handleEqual(eq.attribute(), eq.value());
           } else {
             EqualNullSafe eq = (EqualNullSafe) filter;
             if (eq.value() == null) {
               return isNull(eq.attribute());
             } else {
-              return equal(eq.attribute(), convertLiteral(eq.value()));
+              return handleEqual(eq.attribute(), eq.value());

Review comment:
       Yes, I agree. Rewriting filters should be done in translation to Iceberg so that we have simpler behavior and strong assumptions.




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