karuppayya commented on a change in pull request #3400: URL: https://github.com/apache/iceberg/pull/3400#discussion_r739956697
########## File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java ########## @@ -0,0 +1,300 @@ +/* + * 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.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +public class TestRuntimeFiltering extends SparkCatalogTestBase { + + public TestRuntimeFiltering(String catalogName, String implementation, Map<String, String> config) { + super(catalogName, implementation, config); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS dim"); + } + + @Test + public void testIdentityPartitionedTable() throws NoSuchTableException { + sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", tableName); + + Dataset<Row> df = spark.range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset<Row> dimDF = spark.range(1, 10) + .withColumn("date", expr("DATE '1970-01-02'")) + .select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.date = d.date AND d.id = 1 ORDER BY id", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("date", 1), 3); + + assertEquals("Should have expected rows", + sql("SELECT * FROM %s WHERE date = DATE '1970-01-02' ORDER BY id", tableName), + sql(query)); + } + + @Test + public void testBucketedTable() throws NoSuchTableException { + sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (bucket(8, id))", tableName); + + Dataset<Row> df = spark.range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset<Row> dimDF = spark.range(1, 2) + .withColumn("date", expr("DATE '1970-01-02'")) + .select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.id = d.id AND d.date = DATE '1970-01-02' ORDER BY date", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("id", 1), 7); + + assertEquals("Should have expected rows", + sql("SELECT * FROM %s WHERE id = 1 ORDER BY date", tableName), + sql(query)); + } + + @Test + public void testBucketedTableWithMultipleSpecs() throws NoSuchTableException { + sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) USING iceberg", tableName); + + Dataset<Row> df1 = spark.range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 2 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df1.coalesce(1).writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + table.updateSpec() + .addField(Expressions.bucket("id", 8)) + .commit(); + + sql("REFRESH TABLE %s", tableName); + + Dataset<Row> df2 = spark.range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df2.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset<Row> dimDF = spark.range(1, 2) + .withColumn("date", expr("DATE '1970-01-02'")) + .select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.id = d.id AND d.date = DATE '1970-01-02' ORDER BY date", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("id", 1), 7); + + assertEquals("Should have expected rows", + sql("SELECT * FROM %s WHERE id = 1 ORDER BY date", tableName), + sql(query)); + } + + @Test + public void testSourceColumnWithDots() throws NoSuchTableException { + sql("CREATE TABLE %s (`i.d` BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (bucket(8, `i.d`))", tableName); + + Dataset<Row> df = spark.range(1, 100) + .withColumnRenamed("id", "i.d") + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(`i.d` % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("`i.d`", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("SELECT * FROM %s WHERE `i.d` = 1", tableName); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset<Row> dimDF = spark.range(1, 2) + .withColumn("date", expr("DATE '1970-01-02'")) + .select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.`i.d` = d.id AND d.date = DATE '1970-01-02' ORDER BY date", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("i.d", 1), 7); + + sql(query); + + assertEquals("Should have expected rows", + sql("SELECT * FROM %s WHERE `i.d` = 1 ORDER BY date", tableName), + sql(query)); + } + + @Test + public void testSourceColumnWithBackticks() throws NoSuchTableException { + sql("CREATE TABLE %s (`i``d` BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (bucket(8, `i``d`))", tableName); + + Dataset<Row> df = spark.range(1, 100) + .withColumnRenamed("id", "i`d") + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(`i``d` % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("`i``d`", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset<Row> dimDF = spark.range(1, 2) + .withColumn("date", expr("DATE '1970-01-02'")) + .select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.`i``d` = d.id AND d.date = DATE '1970-01-02' ORDER BY date", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("i`d", 1), 7); + + assertEquals("Should have expected rows", + sql("SELECT * FROM %s WHERE `i``d` = 1 ORDER BY date", tableName), + sql(query)); + } + + @Test + public void testUnpartitionedTable() throws NoSuchTableException { + sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) USING iceberg", tableName); + + Dataset<Row> df = spark.range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset<Row> dimDF = spark.range(1, 2) + .withColumn("date", expr("DATE '1970-01-02'")) + .select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + assertEquals("Should have expected rows", Review comment: Should we also check for the absence of `dynamicPruningExpression` ########## File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java ########## @@ -0,0 +1,300 @@ +/* + * 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.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +public class TestRuntimeFiltering extends SparkCatalogTestBase { + + public TestRuntimeFiltering(String catalogName, String implementation, Map<String, String> config) { + super(catalogName, implementation, config); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS dim"); + } + + @Test + public void testIdentityPartitionedTable() throws NoSuchTableException { + sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", tableName); + + Dataset<Row> df = spark.range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset<Row> dimDF = spark.range(1, 10) + .withColumn("date", expr("DATE '1970-01-02'")) + .select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.date = d.date AND d.id = 1 ORDER BY id", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("date", 1), 3); Review comment: `Expressions.equal("date", 1)` does this mean 2nd day from epoch? ########## File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java ########## @@ -0,0 +1,264 @@ +/* + * 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.util.List; +import java.util.Map; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +public class TestRuntimeFiltering extends SparkCatalogTestBase { + + public TestRuntimeFiltering(String catalogName, String implementation, Map<String, String> config) { Review comment: Thanks @aokolnychyi , can we add a code comment to the `deleteNotMatchingFiles` method on the purpose of deletion. ########## File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java ########## @@ -74,60 +101,158 @@ throw new IllegalArgumentException("Cannot only specify option end-snapshot-id to do incremental scan"); } - // look for split behavior overrides in options - this.splitSize = Spark3Util.propertyAsLong(options, SparkReadOptions.SPLIT_SIZE, null); - this.splitLookback = Spark3Util.propertyAsInt(options, SparkReadOptions.LOOKBACK, null); - this.splitOpenFileCost = Spark3Util.propertyAsLong(options, SparkReadOptions.FILE_OPEN_COST, null); + this.splitSize = table instanceof BaseMetadataTable ? readConf.metadataSplitSize() : readConf.splitSize(); + this.splitLookback = readConf.splitLookback(); + this.splitOpenFileCost = readConf.splitOpenFileCost(); + this.runtimeFilterExpressions = Lists.newArrayList(); + } + + private Set<Integer> specIds() { + if (specIds == null) { + Set<Integer> specIdSet = Sets.newHashSet(); + for (FileScanTask file : files()) { + specIdSet.add(file.spec().specId()); + } + this.specIds = specIdSet; + } + + return specIds; + } + + private List<FileScanTask> files() { + if (files == null) { + this.files = planFiles(); + } + + return files; + } + + private List<FileScanTask> planFiles() { + TableScan scan = table() + .newScan() + .option(TableProperties.SPLIT_SIZE, String.valueOf(splitSize)) + .option(TableProperties.SPLIT_LOOKBACK, String.valueOf(splitLookback)) + .option(TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(splitOpenFileCost)) + .caseSensitive(caseSensitive()) + .project(expectedSchema()); + + if (snapshotId != null) { + scan = scan.useSnapshot(snapshotId); + } + + if (asOfTimestamp != null) { + scan = scan.asOfTime(asOfTimestamp); + } + + if (startSnapshotId != null) { + if (endSnapshotId != null) { + scan = scan.appendsBetween(startSnapshotId, endSnapshotId); + } else { + scan = scan.appendsAfter(startSnapshotId); + } + } + + for (Expression filter : filterExpressions()) { + scan = scan.filter(filter); + } + + try (CloseableIterable<FileScanTask> filesIterable = scan.planFiles()) { + return Lists.newArrayList(filesIterable); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close table scan: " + scan, e); + } } @Override protected List<CombinedScanTask> tasks() { if (tasks == null) { - TableScan scan = table() - .newScan() - .caseSensitive(caseSensitive()) - .project(expectedSchema()); + CloseableIterable<FileScanTask> splitFiles = TableScanUtil.splitFiles( + CloseableIterable.withNoopClose(files()), + splitSize); + CloseableIterable<CombinedScanTask> scanTasks = TableScanUtil.planTasks( + splitFiles, splitSize, + splitLookback, splitOpenFileCost); + tasks = Lists.newArrayList(scanTasks); + } - if (snapshotId != null) { - scan = scan.useSnapshot(snapshotId); - } + return tasks; + } - if (asOfTimestamp != null) { - scan = scan.asOfTime(asOfTimestamp); - } + @Override + public NamedReference[] filterAttributes() { + Set<Integer> partitionFieldSourceIds = Sets.newHashSet(); - if (startSnapshotId != null) { - if (endSnapshotId != null) { - scan = scan.appendsBetween(startSnapshotId, endSnapshotId); - } else { - scan = scan.appendsAfter(startSnapshotId); - } + for (Integer specId : specIds()) { + PartitionSpec spec = table().specs().get(specId); + for (PartitionField field : spec.fields()) { + partitionFieldSourceIds.add(field.sourceId()); } + } - if (splitSize != null) { - scan = scan.option(TableProperties.SPLIT_SIZE, splitSize.toString()); - } + Map<Integer, String> quotedNameById = TypeUtil.indexQuotedNameById( + expectedSchema().asStruct(), + name -> String.format("`%s`", name.replace("`", "``"))); - if (splitLookback != null) { - scan = scan.option(TableProperties.SPLIT_LOOKBACK, splitLookback.toString()); - } + return partitionFieldSourceIds.stream() + .filter(fieldId -> expectedSchema().findField(fieldId) != null) + .map(fieldId -> Spark3Util.toNamedReference(quotedNameById.get(fieldId))) + .toArray(NamedReference[]::new); + } - if (splitOpenFileCost != null) { - scan = scan.option(TableProperties.SPLIT_OPEN_FILE_COST, splitOpenFileCost.toString()); + @Override + public void filter(Filter[] filters) { + Expression runtimeFilterExpr = convertRuntimeFilters(filters); + + if (runtimeFilterExpr != Expressions.alwaysTrue()) { + Map<Integer, Evaluator> evaluatorsBySpecId = Maps.newHashMap(); + + for (Integer specId : specIds()) { + PartitionSpec spec = table().specs().get(specId); + Expression inclusiveExpr = Projections.inclusive(spec).project(runtimeFilterExpr); + Evaluator inclusive = new Evaluator(spec.partitionType(), inclusiveExpr); + evaluatorsBySpecId.put(specId, inclusive); } - for (Expression filter : filterExpressions()) { - scan = scan.filter(filter); + LOG.info("Trying to filter {} files using runtime filter {}", files().size(), runtimeFilterExpr); + + List<FileScanTask> filteredFiles = files().stream() + .filter(file -> { + Evaluator evaluator = evaluatorsBySpecId.get(file.spec().specId()); + return evaluator.eval(file.file().partition()); + }) + .collect(Collectors.toList()); + + LOG.info("{}/{} files matched runtime filter {}", filteredFiles.size(), files().size(), runtimeFilterExpr); Review comment: 👍 Would be a very nice information to display on Spark UI ########## File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java ########## @@ -0,0 +1,300 @@ +/* + * 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.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +public class TestRuntimeFiltering extends SparkCatalogTestBase { + + public TestRuntimeFiltering(String catalogName, String implementation, Map<String, String> config) { + super(catalogName, implementation, config); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS dim"); + } + + @Test + public void testIdentityPartitionedTable() throws NoSuchTableException { + sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", tableName); + + Dataset<Row> df = spark.range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset<Row> dimDF = spark.range(1, 10) Review comment: nit: Could see code duplicated for `Dataset<Row> df` and `Dataset<Row> dimDF`, should we init them in `beforeAll` ? ########## File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java ########## @@ -0,0 +1,300 @@ +/* + * 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.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +public class TestRuntimeFiltering extends SparkCatalogTestBase { + + public TestRuntimeFiltering(String catalogName, String implementation, Map<String, String> config) { + super(catalogName, implementation, config); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS dim"); + } + + @Test + public void testIdentityPartitionedTable() throws NoSuchTableException { + sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", tableName); + + Dataset<Row> df = spark.range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset<Row> dimDF = spark.range(1, 10) + .withColumn("date", expr("DATE '1970-01-02'")) + .select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.date = d.date AND d.id = 1 ORDER BY id", Review comment: Should we test for different types(OUTER, SEMI, ANTI etc) of join and make sure dynamic filtering happens/doesnt happen based on type of join? ########## File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java ########## @@ -0,0 +1,300 @@ +/* + * 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.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +public class TestRuntimeFiltering extends SparkCatalogTestBase { + + public TestRuntimeFiltering(String catalogName, String implementation, Map<String, String> config) { + super(catalogName, implementation, config); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS dim"); + } + + @Test + public void testIdentityPartitionedTable() throws NoSuchTableException { + sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", tableName); + + Dataset<Row> df = spark.range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset<Row> dimDF = spark.range(1, 10) + .withColumn("date", expr("DATE '1970-01-02'")) + .select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.date = d.date AND d.id = 1 ORDER BY id", Review comment: Should we add a test for a three way join? like (a join b) join c on a.col1 = b.col1 and a.partcol = c.col2 and c.col3 = value ? In this case, dynamic partition pruning should happen? -- 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
