From c5a30b97832b8943e890727055926b65b24c934d Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 23 Dec 2025 12:21:25 -0800 Subject: [PATCH] Spark 4.1: Migrate to new version framework in DSv2 --- .../org/apache/iceberg/BaseMetadataTable.java | 16 + .../IcebergSparkSessionExtensions.scala | 2 + .../sql/catalyst/analysis/ResolveBranch.scala | 118 ++++ .../extensions/TestCopyOnWriteDelete.java | 108 --- .../extensions/TestCopyOnWriteMerge.java | 106 --- .../extensions/TestCopyOnWriteUpdate.java | 106 --- .../extensions/TestMergeOnReadDelete.java | 2 +- .../TestPartitionedWritesToWapBranch.java | 44 +- .../iceberg/spark/source/PlanUtils.java | 2 +- .../org/apache/iceberg/spark/Spark3Util.java | 42 +- .../apache/iceberg/spark/SparkCatalog.java | 233 ++----- .../apache/iceberg/spark/SparkReadConf.java | 41 +- .../iceberg/spark/SparkReadOptions.java | 16 +- .../iceberg/spark/SparkSessionCatalog.java | 4 +- .../apache/iceberg/spark/SparkTableUtil.java | 34 +- .../apache/iceberg/spark/SparkWriteConf.java | 13 +- .../org/apache/iceberg/spark/TimeTravel.java | 64 ++ .../spark/source/BaseSparkScanBuilder.java | 261 +++++++ .../iceberg/spark/source/BatchDataReader.java | 2 +- .../spark/source/ChangelogRowReader.java | 2 +- .../iceberg/spark/source/IcebergSource.java | 150 ++-- .../source/PositionDeletesRowReader.java | 2 +- .../iceberg/spark/source/RowDataReader.java | 2 +- .../iceberg/spark/source/SparkBatch.java | 14 +- .../spark/source/SparkBatchQueryScan.java | 238 +------ .../spark/source/SparkChangelogScan.java | 9 +- .../source/SparkChangelogScanBuilder.java | 152 ++++ .../spark/source/SparkChangelogTable.java | 48 +- .../source/SparkCopyOnWriteOperation.java | 20 +- .../spark/source/SparkCopyOnWriteScan.java | 55 +- .../spark/source/SparkIncrementalScan.java | 97 +++ .../spark/source/SparkInputPartition.java | 16 +- .../source/SparkPartitioningAwareScan.java | 11 +- .../source/SparkPositionDeltaOperation.java | 22 +- .../spark/source/SparkPositionDeltaWrite.java | 18 +- .../SparkPositionDeltaWriteBuilder.java | 8 +- .../source/SparkRewriteWriteBuilder.java | 1 + .../source/SparkRowLevelOperationBuilder.java | 23 +- .../source/SparkRuntimeFilterableScan.java | 200 ++++++ .../iceberg/spark/source/SparkScan.java | 62 +- .../spark/source/SparkScanBuilder.java | 656 ++++-------------- .../iceberg/spark/source/SparkStagedScan.java | 14 +- .../spark/source/SparkStagedScanBuilder.java | 61 +- .../iceberg/spark/source/SparkTable.java | 352 ++++------ .../iceberg/spark/source/SparkWrite.java | 21 +- .../spark/source/SparkWriteBuilder.java | 234 ++++--- .../spark/source/StagedSparkTable.java | 2 +- .../actions/TestRemoveOrphanFilesAction.java | 3 +- .../spark/source/TestDataSourceOptions.java | 21 +- .../spark/source/TestFilteredScan.java | 21 +- .../source/TestForwardCompatibility.java | 2 + .../source/TestIcebergSourceTablesBase.java | 20 +- .../spark/source/TestSnapshotSelection.java | 75 +- .../spark/source/TestSparkCatalog.java | 2 +- .../TestSparkCatalogCacheExpiration.java | 39 +- .../spark/source/TestSparkDataWrite.java | 2 +- .../iceberg/spark/source/TestSparkScan.java | 48 +- .../iceberg/spark/sql/TestDeleteFrom.java | 2 +- .../apache/iceberg/spark/sql/TestSelect.java | 37 +- .../sql/UnpartitionedWritesTestBase.java | 4 +- 60 files changed, 1936 insertions(+), 2044 deletions(-) create mode 100644 spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveBranch.scala rename spark/v4.1/{spark/src/test/java/org/apache/iceberg/spark/sql => spark-extensions/src/test/java/org/apache/iceberg/spark/extensions}/TestPartitionedWritesToWapBranch.java (66%) create mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/TimeTravel.java create mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseSparkScanBuilder.java create mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScanBuilder.java create mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkIncrementalScan.java create mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRuntimeFilterableScan.java diff --git a/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java b/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java index 9f3113f2df8e..420a50be8ab0 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; import org.apache.iceberg.encryption.EncryptionManager; @@ -29,6 +30,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.transforms.Transforms; /** @@ -39,6 +41,16 @@ * needed when reading the table data after deserialization. */ public abstract class BaseMetadataTable extends BaseReadOnlyTable implements Serializable { + private static final Set TIME_TRAVEL_TABLE_TYPES = + ImmutableSet.of( + MetadataTableType.ENTRIES, + MetadataTableType.FILES, + MetadataTableType.DATA_FILES, + MetadataTableType.DELETE_FILES, + MetadataTableType.MANIFESTS, + MetadataTableType.PARTITIONS, + MetadataTableType.POSITION_DELETES); + private final PartitionSpec spec = PartitionSpec.unpartitioned(); private final SortOrder sortOrder = SortOrder.unsorted(); private final BaseTable table; @@ -106,6 +118,10 @@ static Map transformSpecs( abstract MetadataTableType metadataTableType(); + public boolean supportsTimeTravel() { + return TIME_TRAVEL_TABLE_TYPES.contains(metadataTableType()); + } + public BaseTable table() { return table; } diff --git a/spark/v4.1/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala index c4de35010c6e..81824e05e92d 100644 --- a/spark/v4.1/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala +++ b/spark/v4.1/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala @@ -20,6 +20,7 @@ package org.apache.iceberg.spark.extensions import org.apache.spark.sql.SparkSessionExtensions import org.apache.spark.sql.catalyst.analysis.CheckViews +import org.apache.spark.sql.catalyst.analysis.ResolveBranch import org.apache.spark.sql.catalyst.analysis.ResolveViews import org.apache.spark.sql.catalyst.optimizer.ReplaceStaticInvoke import org.apache.spark.sql.catalyst.parser.extensions.IcebergSparkSqlExtensionsParser @@ -33,6 +34,7 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) { // analyzer extensions extensions.injectResolutionRule { spark => ResolveViews(spark) } + extensions.injectPostHocResolutionRule { spark => ResolveBranch(spark) } extensions.injectCheckRule(_ => CheckViews) // optimizer extensions diff --git a/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveBranch.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveBranch.scala new file mode 100644 index 000000000000..9cc2c188a06c --- /dev/null +++ b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveBranch.scala @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.spark.sql.catalyst.analysis + +import org.apache.iceberg.spark.PathIdentifier +import org.apache.iceberg.spark.SparkTableUtil +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.RowLevelWrite +import org.apache.spark.sql.catalyst.plans.logical.V2WriteCommand +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.write.RowLevelOperation +import org.apache.spark.sql.connector.write.RowLevelOperationInfoImpl +import org.apache.spark.sql.connector.write.RowLevelOperationTable +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.ExtractV2Table +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +/** + * A rule that resolves the target branch for Iceberg reads and writes. + *

+ * The branch must be determined and pinned during analysis. The current DSv2 framework + * doesn't provide access to all necessary options during the initial table loading, + * forcing us to finalize the branch selection in a custom analyzer rule. Future Spark + * versions will have a built-in mechanism to cleanly determine the target branch. + */ +case class ResolveBranch(spark: SparkSession) extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + // row-level operations like DELETE, UPDATE, and MERGE + case w @ IcebergRowLevelWrite(table, operation, options) => + val branch = SparkTableUtil.determineWriteBranch(spark, table, options) + if (table.branch != branch) { + val newTable = table.copyWithBranch(branch) + val info = RowLevelOperationInfoImpl(operation.command, options) + val newOperation = newTable.newRowLevelOperationBuilder(info).build() + val newOperationTable = RowLevelOperationTable(newTable, newOperation) + val newTarget = transformPreservingType(w.table) { + case r @ ExtractV2Table(RowLevelOperationTable(_, _)) => r.copy(table = newOperationTable) + } + val newQuery = transformPreservingType(w.query) { + case r @ ExtractV2Table(RowLevelOperationTable(_, _)) => r.copy(table = newOperationTable) + } + w.withNewTable(newTarget).withNewQuery(newQuery) + } else { + w + } + + // batch write operations like append or overwrite + case w: V2WriteCommand => + val newTarget = transformPreservingType(w.table) { + case r @ DataSourceV2Relation(table: SparkTable, _, _, _, options, _) => + val branch = SparkTableUtil.determineWriteBranch(spark, table, options) + if (table.branch != branch) r.copy(table = table.copyWithBranch(branch)) else r + } + w.withNewTable(newTarget) + + // scan operations + // branch selector is added to identifier to ensure further refreshes point to correct branch + case r @ DataSourceV2Relation(table: SparkTable, _, _, Some(ident), options, None) => + val branch = SparkTableUtil.determineReadBranch(spark, table, options) + if (table.branch != branch) { + val branchSelector = s"branch_$branch" + val newIdent = ident match { + case path: PathIdentifier if path.location.contains("#") => + new PathIdentifier(path.location + "," + branchSelector) + case path: PathIdentifier => + new PathIdentifier(path.location + "#" + branchSelector) + case _ => + Identifier.of(ident.namespace :+ ident.name, branchSelector) + } + r.copy(table = table.copyWithBranch(branch), identifier = Some(newIdent)) + } else { + r + } + } + + private def transformPreservingType[T <: LogicalPlan](plan: T)( + func: PartialFunction[LogicalPlan, LogicalPlan]): T = { + plan.transform(func).asInstanceOf[T] + } +} + +// Iceberg specific extractor for row-level operations like DELETE, UPDATE, and MERGE +private object IcebergRowLevelWrite { + def unapply( + write: RowLevelWrite): Option[(SparkTable, RowLevelOperation, CaseInsensitiveStringMap)] = { + EliminateSubqueryAliases(write.table) match { + case DataSourceV2Relation( + RowLevelOperationTable(table: SparkTable, operation), + _, + _, + _, + options, + _) => + Some((table, operation, options)) + case _ => None + } + } +} diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java index f7ded0c4d7d2..e0b852a9d0fd 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java @@ -18,26 +18,12 @@ */ package org.apache.iceberg.spark.extensions; -import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Files; import org.apache.iceberg.ParameterizedTestExtension; @@ -52,14 +38,10 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; -import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.util.SnapshotUtil; -import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; -import org.awaitility.Awaitility; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -72,96 +54,6 @@ protected Map extraTableProperties() { TableProperties.DELETE_MODE, RowLevelOperationMode.COPY_ON_WRITE.modeName()); } - @TestTemplate - public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception { - // this test can only be run with Hive tables as it requires a reliable lock - // also, the table cache must be enabled so that the same table instance can be reused - assumeThat(catalogName).isEqualToIgnoringCase("testhive"); - - createAndInitUnpartitionedTable(); - createOrReplaceView("deleted_id", Collections.singletonList(1), Encoders.INT()); - - sql( - "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", - tableName, DELETE_ISOLATION_LEVEL, "snapshot"); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); - createBranchIfNeeded(); - - Table table = Spark3Util.loadIcebergTable(spark, tableName); - - ExecutorService executorService = - MoreExecutors.getExitingExecutorService( - (ThreadPoolExecutor) Executors.newFixedThreadPool(2)); - - AtomicInteger barrier = new AtomicInteger(0); - AtomicBoolean shouldAppend = new AtomicBoolean(true); - - // delete thread - Future deleteFuture = - executorService.submit( - () -> { - for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - int currentNumOperations = numOperations; - Awaitility.await() - .pollInterval(10, TimeUnit.MILLISECONDS) - .atMost(5, TimeUnit.SECONDS) - .until(() -> barrier.get() >= currentNumOperations * 2); - - sql("DELETE FROM %s WHERE id IN (SELECT * FROM deleted_id)", commitTarget()); - - barrier.incrementAndGet(); - } - }); - - // append thread - Future appendFuture = - executorService.submit( - () -> { - GenericRecord record = GenericRecord.create(SnapshotUtil.schemaFor(table, branch)); - record.set(0, 1); // id - record.set(1, "hr"); // dep - - for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - int currentNumOperations = numOperations; - Awaitility.await() - .pollInterval(10, TimeUnit.MILLISECONDS) - .atMost(5, TimeUnit.SECONDS) - .until(() -> !shouldAppend.get() || barrier.get() >= currentNumOperations * 2); - - if (!shouldAppend.get()) { - return; - } - - for (int numAppends = 0; numAppends < 5; numAppends++) { - DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); - AppendFiles appendFiles = table.newFastAppend().appendFile(dataFile); - if (branch != null) { - appendFiles.toBranch(branch); - } - - appendFiles.commit(); - } - - barrier.incrementAndGet(); - } - }); - - try { - assertThatThrownBy(deleteFuture::get) - .isInstanceOf(ExecutionException.class) - .cause() - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("the table has been concurrently modified"); - } finally { - shouldAppend.set(false); - appendFuture.cancel(true); - } - - executorService.shutdown(); - assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); - } - @TestTemplate public void testRuntimeFilteringWithPreservedDataGrouping() throws NoSuchTableException { createAndInitPartitionedTable(); diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java index fef8b28c689a..36b7c5818ef6 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java @@ -18,37 +18,21 @@ */ package org.apache.iceberg.spark.extensions; -import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Collections; import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.iceberg.DataFile; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; -import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.internal.SQLConf; -import org.awaitility.Awaitility; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -61,96 +45,6 @@ protected Map extraTableProperties() { TableProperties.MERGE_MODE, RowLevelOperationMode.COPY_ON_WRITE.modeName()); } - @TestTemplate - public synchronized void testMergeWithConcurrentTableRefresh() throws Exception { - // this test can only be run with Hive tables as it requires a reliable lock - // also, the table cache must be enabled so that the same table instance can be reused - assumeThat(catalogName).isEqualToIgnoringCase("testhive"); - - createAndInitTable("id INT, dep STRING"); - createOrReplaceView("source", Collections.singletonList(1), Encoders.INT()); - - sql( - "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", - tableName, MERGE_ISOLATION_LEVEL, "snapshot"); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); - createBranchIfNeeded(); - - Table table = Spark3Util.loadIcebergTable(spark, tableName); - - ExecutorService executorService = - MoreExecutors.getExitingExecutorService( - (ThreadPoolExecutor) Executors.newFixedThreadPool(2)); - - AtomicInteger barrier = new AtomicInteger(0); - AtomicBoolean shouldAppend = new AtomicBoolean(true); - - // merge thread - Future mergeFuture = - executorService.submit( - () -> { - for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - int currentNumOperations = numOperations; - Awaitility.await() - .pollInterval(10, TimeUnit.MILLISECONDS) - .atMost(5, TimeUnit.SECONDS) - .until(() -> barrier.get() >= currentNumOperations * 2); - - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.value " - + "WHEN MATCHED THEN " - + " UPDATE SET dep = 'x'", - tableName); - - barrier.incrementAndGet(); - } - }); - - // append thread - Future appendFuture = - executorService.submit( - () -> { - GenericRecord record = GenericRecord.create(table.schema()); - record.set(0, 1); // id - record.set(1, "hr"); // dep - - for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - int currentNumOperations = numOperations; - Awaitility.await() - .pollInterval(10, TimeUnit.MILLISECONDS) - .atMost(5, TimeUnit.SECONDS) - .until(() -> !shouldAppend.get() || barrier.get() >= currentNumOperations * 2); - - if (!shouldAppend.get()) { - return; - } - - for (int numAppends = 0; numAppends < 5; numAppends++) { - DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); - table.newFastAppend().appendFile(dataFile).commit(); - } - - barrier.incrementAndGet(); - } - }); - - try { - assertThatThrownBy(mergeFuture::get) - .isInstanceOf(ExecutionException.class) - .cause() - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("the table has been concurrently modified"); - } finally { - shouldAppend.set(false); - appendFuture.cancel(true); - } - - executorService.shutdown(); - assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); - } - @TestTemplate public void testRuntimeFilteringWithReportedPartitioning() { createAndInitTable("id INT, dep STRING"); diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java index 21d1377b2b98..ab5b6da50bcf 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java @@ -18,36 +18,19 @@ */ package org.apache.iceberg.spark.extensions; -import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; -import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.internal.SQLConf; -import org.awaitility.Awaitility; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -60,95 +43,6 @@ protected Map extraTableProperties() { TableProperties.UPDATE_MODE, RowLevelOperationMode.COPY_ON_WRITE.modeName()); } - @TestTemplate - public synchronized void testUpdateWithConcurrentTableRefresh() throws Exception { - // this test can only be run with Hive tables as it requires a reliable lock - // also, the table cache must be enabled so that the same table instance can be reused - assumeThat(catalogName).isEqualToIgnoringCase("testhive"); - - createAndInitTable("id INT, dep STRING"); - - sql( - "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", - tableName, UPDATE_ISOLATION_LEVEL, "snapshot"); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); - createBranchIfNeeded(); - - Table table = Spark3Util.loadIcebergTable(spark, tableName); - - ExecutorService executorService = - MoreExecutors.getExitingExecutorService( - (ThreadPoolExecutor) Executors.newFixedThreadPool(2)); - - AtomicInteger barrier = new AtomicInteger(0); - AtomicBoolean shouldAppend = new AtomicBoolean(true); - - // update thread - Future updateFuture = - executorService.submit( - () -> { - for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - int currentNumOperations = numOperations; - Awaitility.await() - .pollInterval(10, TimeUnit.MILLISECONDS) - .atMost(5, TimeUnit.SECONDS) - .until(() -> barrier.get() >= currentNumOperations * 2); - - sql("UPDATE %s SET id = -1 WHERE id = 1", commitTarget()); - - barrier.incrementAndGet(); - } - }); - - // append thread - Future appendFuture = - executorService.submit( - () -> { - GenericRecord record = GenericRecord.create(SnapshotUtil.schemaFor(table, branch)); - record.set(0, 1); // id - record.set(1, "hr"); // dep - - for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - int currentNumOperations = numOperations; - Awaitility.await() - .pollInterval(10, TimeUnit.MILLISECONDS) - .atMost(5, TimeUnit.SECONDS) - .until(() -> !shouldAppend.get() || barrier.get() >= currentNumOperations * 2); - - if (!shouldAppend.get()) { - return; - } - - for (int numAppends = 0; numAppends < 5; numAppends++) { - DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); - AppendFiles appendFiles = table.newFastAppend().appendFile(dataFile); - if (branch != null) { - appendFiles.toBranch(branch); - } - - appendFiles.commit(); - } - - barrier.incrementAndGet(); - } - }); - - try { - assertThatThrownBy(updateFuture::get) - .isInstanceOf(ExecutionException.class) - .cause() - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("the table has been concurrently modified"); - } finally { - shouldAppend.set(false); - appendFuture.cancel(true); - } - - executorService.shutdown(); - assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); - } - @TestTemplate public void testRuntimeFilteringWithReportedPartitioning() { createAndInitTable("id INT, dep STRING"); diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index bf9c53f82dbd..1f69fe812588 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -294,7 +294,7 @@ public void testCommitUnknownException() { Table spyTable = spy(table); when(spyTable.newRowDelta()).thenReturn(spyNewRowDelta); SparkTable sparkTable = - branch == null ? new SparkTable(spyTable, false) : new SparkTable(spyTable, branch, false); + branch == null ? new SparkTable(spyTable) : SparkTable.create(spyTable, branch); ImmutableMap config = ImmutableMap.of( diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPartitionedWritesToWapBranch.java similarity index 66% rename from spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPartitionedWritesToWapBranch.java index 1b6334f23e50..1db18f3a857d 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPartitionedWritesToWapBranch.java @@ -16,18 +16,31 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.spark.sql; +package org.apache.iceberg.spark.extensions; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.net.InetAddress; import java.util.List; +import java.util.Map; import java.util.UUID; +import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.spark.SparkSQLProperties; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.spark.sql.PartitionedWritesTestBase; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -37,6 +50,35 @@ public class TestPartitionedWritesToWapBranch extends PartitionedWritesTestBase private static final String BRANCH = "test"; + @BeforeAll + public static void startMetastoreAndSpark() { + TestBase.metastore = new TestHiveMetastore(); + metastore.start(); + TestBase.hiveConf = metastore.hiveConf(); + + TestBase.spark.stop(); + + TestBase.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .config("spark.testing", "true") + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) + .config("spark.sql.shuffle.partitions", "4") + .config("spark.sql.hive.metastorePartitionPruningFallbackOnException", "true") + .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") + .enableHiveSupport() + .getOrCreate(); + + TestBase.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + String className = HiveCatalog.class.getName(); + Map options = ImmutableMap.of(); + TestBase.catalog = (HiveCatalog) CatalogUtil.loadCatalog(className, "hive", options, hiveConf); + } + @BeforeEach @Override public void createTables() { diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java index 148717e14255..781b39a9abf0 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java @@ -49,7 +49,7 @@ public static List collectPushDownFil } SparkBatchQueryScan batchQueryScan = (SparkBatchQueryScan) scanRelation.scan(); - return batchQueryScan.filterExpressions().stream(); + return Stream.of(batchQueryScan.filter()); }) .collect(Collectors.toList()); } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java index df42175c3476..7359aec371a4 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java @@ -109,14 +109,6 @@ public class Spark3Util { private Spark3Util() {} - public static CaseInsensitiveStringMap setOption( - String key, String value, CaseInsensitiveStringMap options) { - Map newOptions = Maps.newHashMap(); - newOptions.putAll(options); - newOptions.put(key, value); - return new CaseInsensitiveStringMap(newOptions); - } - public static Map rebuildCreateProperties(Map createProperties) { ImmutableMap.Builder tableProperties = ImmutableMap.builder(); createProperties.entrySet().stream() @@ -546,6 +538,31 @@ public static boolean extensionsEnabled(SparkSession spark) { return extensions.contains("IcebergSparkSessionExtensions"); } + public static boolean containsIncrementalOptions(CaseInsensitiveStringMap options) { + return options.containsKey(SparkReadOptions.START_SNAPSHOT_ID) + || options.containsKey(SparkReadOptions.END_SNAPSHOT_ID); + } + + public static void validateNoLegacyTimeTravel(CaseInsensitiveStringMap options) { + Preconditions.checkArgument( + !options.containsKey(SparkReadOptions.LEGACY_SNAPSHOT_ID), + "Time travel option `%s` is no longer supported, use Spark built-in `%s` instead", + SparkReadOptions.LEGACY_SNAPSHOT_ID, + SparkReadOptions.VERSION_AS_OF); + + Preconditions.checkArgument( + !options.containsKey(SparkReadOptions.LEGACY_AS_OF_TIMESTAMP), + "Time travel option `%s` (in millis) is no longer supported, use Spark built-in `%s` instead (properly formatted timestamp)", + SparkReadOptions.LEGACY_AS_OF_TIMESTAMP, + SparkReadOptions.TIMESTAMP_AS_OF); + + Preconditions.checkArgument( + !options.containsKey(SparkReadOptions.LEGACY_TAG), + "Time travel option `%s` is no longer supported, use Spark built-in `%s` instead", + SparkReadOptions.LEGACY_TAG, + SparkReadOptions.VERSION_AS_OF); + } + public static class DescribeSchemaVisitor extends TypeUtil.SchemaVisitor { private static final Joiner COMMA = Joiner.on(','); private static final DescribeSchemaVisitor INSTANCE = new DescribeSchemaVisitor(); @@ -850,6 +867,15 @@ public CatalogAndIdentifier(Pair identifier) { this.identifier = identifier.second(); } + public TableCatalog tableCatalog() { + Preconditions.checkArgument( + catalog instanceof TableCatalog, + "%s is not a valid table catalog: %s", + catalog.name(), + catalog.getClass().getName()); + return (TableCatalog) catalog; + } + public CatalogPlugin catalog() { return catalog; } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index da22607d05b0..12eefe697348 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -24,13 +24,10 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.TreeMap; -import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; -import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CachingCatalog; import org.apache.iceberg.CatalogProperties; @@ -39,8 +36,6 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -67,7 +62,6 @@ import org.apache.iceberg.spark.source.StagedSparkTable; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.view.UpdateViewProperties; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; @@ -129,11 +123,9 @@ public class SparkCatalog extends BaseCatalog { private static final Pattern SNAPSHOT_ID = Pattern.compile("snapshot_id_(\\d+)"); private static final Pattern BRANCH = Pattern.compile("branch_(.*)"); private static final Pattern TAG = Pattern.compile("tag_(.*)"); - private static final String REWRITE = "rewrite"; private String catalogName = null; private Catalog icebergCatalog = null; - private boolean cacheEnabled = CatalogProperties.CACHE_ENABLED_DEFAULT; private SupportsNamespaces asNamespaceCatalog = null; private ViewCatalog asViewCatalog = null; private String[] defaultNamespace = null; @@ -167,71 +159,17 @@ protected TableIdentifier buildIdentifier(Identifier identifier) { @Override public Table loadTable(Identifier ident) throws NoSuchTableException { - try { - return load(ident); - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - throw new NoSuchTableException(ident); - } + return load(ident, null /* no time travel */); } @Override public Table loadTable(Identifier ident, String version) throws NoSuchTableException { - Table table = loadTable(ident); - - if (table instanceof SparkTable) { - SparkTable sparkTable = (SparkTable) table; - - Preconditions.checkArgument( - sparkTable.snapshotId() == null && sparkTable.branch() == null, - "Cannot do time-travel based on both table identifier and AS OF"); - - try { - return sparkTable.copyWithSnapshotId(Long.parseLong(version)); - } catch (NumberFormatException e) { - SnapshotRef ref = sparkTable.table().refs().get(version); - ValidationException.check( - ref != null, - "Cannot find matching snapshot ID or reference name for version %s", - version); - - if (ref.isBranch()) { - return sparkTable.copyWithBranch(version); - } else { - return sparkTable.copyWithSnapshotId(ref.snapshotId()); - } - } - - } else if (table instanceof SparkChangelogTable) { - throw new UnsupportedOperationException("AS OF is not supported for changelogs"); - - } else { - throw new IllegalArgumentException("Unknown Spark table type: " + table.getClass().getName()); - } + return load(ident, TimeTravel.version(version)); } @Override - public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableException { - Table table = loadTable(ident); - - if (table instanceof SparkTable) { - SparkTable sparkTable = (SparkTable) table; - - Preconditions.checkArgument( - sparkTable.snapshotId() == null && sparkTable.branch() == null, - "Cannot do time-travel based on both table identifier and AS OF"); - - // convert the timestamp to milliseconds as Spark passes microseconds - // but Iceberg uses milliseconds for snapshot timestamps - long timestampMillis = TimeUnit.MICROSECONDS.toMillis(timestamp); - long snapshotId = SnapshotUtil.snapshotIdAsOfTime(sparkTable.table(), timestampMillis); - return sparkTable.copyWithSnapshotId(snapshotId); - - } else if (table instanceof SparkChangelogTable) { - throw new UnsupportedOperationException("AS OF is not supported for changelogs"); - - } else { - throw new IllegalArgumentException("Unknown Spark table type: " + table.getClass().getName()); - } + public Table loadTable(Identifier ident, long timestampMicros) throws NoSuchTableException { + return load(ident, TimeTravel.timestampMicros(timestampMicros)); } @Override @@ -256,7 +194,7 @@ public Table createTable( .withLocation(properties.get("location")) .withProperties(Spark3Util.rebuildCreateProperties(properties)) .create(); - return new SparkTable(icebergTable, !cacheEnabled); + return new SparkTable(icebergTable); } catch (AlreadyExistsException e) { throw new TableAlreadyExistsException(ident); } @@ -355,7 +293,7 @@ public Table alterTable(Identifier ident, TableChange... changes) throws NoSuchT org.apache.iceberg.Table table = icebergCatalog.loadTable(buildIdentifier(ident)); commitChanges( table, setLocation, setSnapshotId, pickSnapshotId, propertyChanges, schemaChanges); - return new SparkTable(table, true /* refreshEagerly */); + return new SparkTable(table); } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { throw new NoSuchTableException(ident); } @@ -748,7 +686,7 @@ public void renameView(Identifier fromIdentifier, Identifier toIdentifier) public final void initialize(String name, CaseInsensitiveStringMap options) { super.initialize(name, options); - this.cacheEnabled = + boolean cacheEnabled = PropertyUtil.propertyAsBoolean( options, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); @@ -767,7 +705,7 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { // An expiration interval of 0ms effectively disables caching. // Do not wrap with CachingCatalog. if (cacheExpirationIntervalMs == 0) { - this.cacheEnabled = false; + cacheEnabled = false; } Catalog catalog = buildIcebergCatalog(name, options); @@ -858,18 +796,18 @@ private static void checkNotPathIdentifier(Identifier identifier, String method) } } - private Table load(Identifier ident) { + private Table load(Identifier ident, TimeTravel timeTravel) throws NoSuchTableException { if (isPathIdentifier(ident)) { - return loadFromPathIdentifier((PathIdentifier) ident); + return loadPath((PathIdentifier) ident, timeTravel); } try { org.apache.iceberg.Table table = icebergCatalog.loadTable(buildIdentifier(ident)); - return new SparkTable(table, !cacheEnabled); + return SparkTable.create(table, timeTravel); } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { if (ident.namespace().length == 0) { - throw e; + throw new NoSuchTableException(ident); } // if the original load didn't work, try using the namespace as an identifier because @@ -880,50 +818,35 @@ private Table load(Identifier ident) { table = icebergCatalog.loadTable(namespaceAsIdent); } catch (Exception ignored) { // the namespace does not identify a table, so it cannot be a table with a snapshot selector - // throw the original exception - throw e; + // throw an exception for the original identifier + throw new NoSuchTableException(ident); } // loading the namespace as a table worked, check the name to see if it is a valid selector // or if the name points to the changelog if (ident.name().equalsIgnoreCase(SparkChangelogTable.TABLE_NAME)) { - return new SparkChangelogTable(table, !cacheEnabled); - } - - Matcher at = AT_TIMESTAMP.matcher(ident.name()); - if (at.matches()) { - long asOfTimestamp = Long.parseLong(at.group(1)); - long snapshotId = SnapshotUtil.snapshotIdAsOfTime(table, asOfTimestamp); - return new SparkTable(table, snapshotId, !cacheEnabled); - } - - Matcher id = SNAPSHOT_ID.matcher(ident.name()); - if (id.matches()) { - long snapshotId = Long.parseLong(id.group(1)); - return new SparkTable(table, snapshotId, !cacheEnabled); + Preconditions.checkArgument(timeTravel == null, "Can't time travel in changelog"); + return new SparkChangelogTable(table); } Matcher branch = BRANCH.matcher(ident.name()); if (branch.matches()) { - return new SparkTable(table, branch.group(1), !cacheEnabled); - } - - Matcher tag = TAG.matcher(ident.name()); - if (tag.matches()) { - Snapshot tagSnapshot = table.snapshot(tag.group(1)); - if (tagSnapshot != null) { - return new SparkTable(table, tagSnapshot.snapshotId(), !cacheEnabled); - } + Preconditions.checkArgument(timeTravel == null, "Can't time travel in branch"); + return SparkTable.create(table, branch.group(1)); } - if (ident.name().equalsIgnoreCase(REWRITE)) { - return new SparkTable(table, null, !cacheEnabled, true); + TimeTravel timeTravelSelector = parseTimeTravelSelector(ident.name()); + if (timeTravelSelector != null) { + Preconditions.checkArgument( + timeTravel == null, + "Can't time travel using selector and Spark time travel spec at the same time"); + return SparkTable.create(table, timeTravelSelector); } // the name wasn't a valid snapshot selector and did not point to the changelog - // throw the original exception - throw e; + // throw an exception for the original identifier + throw new NoSuchTableException(ident); } } @@ -939,16 +862,13 @@ private Pair> parseLocationString(String location) { } @SuppressWarnings("CyclomaticComplexity") - private Table loadFromPathIdentifier(PathIdentifier ident) { + private Table loadPath(PathIdentifier ident, TimeTravel timeTravel) throws NoSuchTableException { Pair> parsed = parseLocationString(ident.location()); String metadataTableName = null; - Long asOfTimestamp = null; - Long snapshotId = null; String branch = null; - String tag = null; + TimeTravel effectiveTimeTravel = timeTravel; boolean isChangelog = false; - boolean isRewrite = false; for (String meta : parsed.second()) { if (meta.equalsIgnoreCase(SparkChangelogTable.TABLE_NAME)) { @@ -956,77 +876,70 @@ private Table loadFromPathIdentifier(PathIdentifier ident) { continue; } - if (REWRITE.equals(meta)) { - isRewrite = true; - continue; - } - if (MetadataTableType.from(meta) != null) { metadataTableName = meta; continue; } - Matcher at = AT_TIMESTAMP.matcher(meta); - if (at.matches()) { - asOfTimestamp = Long.parseLong(at.group(1)); - continue; - } - - Matcher id = SNAPSHOT_ID.matcher(meta); - if (id.matches()) { - snapshotId = Long.parseLong(id.group(1)); - continue; - } - - Matcher branchRef = BRANCH.matcher(meta); - if (branchRef.matches()) { - branch = branchRef.group(1); + Matcher branchSelector = BRANCH.matcher(meta); + if (branchSelector.matches()) { + branch = branchSelector.group(1); continue; } - Matcher tagRef = TAG.matcher(meta); - if (tagRef.matches()) { - tag = tagRef.group(1); + TimeTravel timeTravelSelector = parseTimeTravelSelector(meta); + if (timeTravelSelector != null) { + Preconditions.checkArgument( + timeTravel == null, + "Can't time travel using selector and Spark time travel spec at the same time"); + Preconditions.checkArgument( + effectiveTimeTravel == null, + "Can't time travel using multiple time travel selectors: (%s, %s)", + effectiveTimeTravel, + timeTravelSelector); + effectiveTimeTravel = timeTravelSelector; } } - Preconditions.checkArgument( - Stream.of(snapshotId, asOfTimestamp, branch, tag).filter(Objects::nonNull).count() <= 1, - "Can specify only one of snapshot-id (%s), as-of-timestamp (%s), branch (%s), tag (%s)", - snapshotId, - asOfTimestamp, - branch, - tag); - - Preconditions.checkArgument( - !isChangelog || (snapshotId == null && asOfTimestamp == null), - "Cannot specify snapshot-id and as-of-timestamp for changelogs"); - - org.apache.iceberg.Table table = - tables.load(parsed.first() + (metadataTableName != null ? "#" + metadataTableName : "")); + org.apache.iceberg.Table table; + try { + String qualifier = metadataTableName != null ? "#" + metadataTableName : ""; + table = tables.load(parsed.first() + qualifier); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(ident); + } if (isChangelog) { - return new SparkChangelogTable(table, !cacheEnabled); - - } else if (asOfTimestamp != null) { - long snapshotIdAsOfTime = SnapshotUtil.snapshotIdAsOfTime(table, asOfTimestamp); - return new SparkTable(table, snapshotIdAsOfTime, !cacheEnabled); + Preconditions.checkArgument(branch == null, "Cannot specify branch for changelogs"); + Preconditions.checkArgument(effectiveTimeTravel == null, "Cannot time travel in changelogs"); + return new SparkChangelogTable(table); } else if (branch != null) { - return new SparkTable(table, branch, !cacheEnabled); + Preconditions.checkArgument(effectiveTimeTravel == null, "Cannot time travel in branch"); + return SparkTable.create(table, branch); + + } else { + return SparkTable.create(table, effectiveTimeTravel); + } + } - } else if (tag != null) { - Snapshot tagSnapshot = table.snapshot(tag); - Preconditions.checkArgument( - tagSnapshot != null, "Cannot find snapshot associated with tag name: %s", tag); - return new SparkTable(table, tagSnapshot.snapshotId(), !cacheEnabled); + private TimeTravel parseTimeTravelSelector(String selector) { + Matcher at = AT_TIMESTAMP.matcher(selector); + if (at.matches()) { + return TimeTravel.timestampMillis(Long.parseLong(at.group(1))); + } - } else if (isRewrite) { - return new SparkTable(table, null, !cacheEnabled, true); + Matcher id = SNAPSHOT_ID.matcher(selector); + if (id.matches()) { + return TimeTravel.version(id.group(1)); + } - } else { - return new SparkTable(table, snapshotId, !cacheEnabled); + Matcher tag = TAG.matcher(selector); + if (tag.matches()) { + return TimeTravel.version(tag.group(1)); } + + return null; } private Identifier namespaceToIdentifier(String[] namespace) { diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index ec56e5f23922..f63e81e7677e 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -25,6 +25,8 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.Pair; import org.apache.spark.SparkConf; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -90,14 +92,6 @@ public boolean localityEnabled() { .parse(); } - public Long snapshotId() { - return confParser.longConf().option(SparkReadOptions.SNAPSHOT_ID).parseOptional(); - } - - public Long asOfTimestamp() { - return confParser.longConf().option(SparkReadOptions.AS_OF_TIMESTAMP).parseOptional(); - } - public Long startSnapshotId() { return confParser.longConf().option(SparkReadOptions.START_SNAPSHOT_ID).parseOptional(); } @@ -106,14 +100,6 @@ public Long endSnapshotId() { return confParser.longConf().option(SparkReadOptions.END_SNAPSHOT_ID).parseOptional(); } - public String branch() { - return SparkTableUtil.determineReadBranch(spark, table, branch, options); - } - - public String tag() { - return confParser.stringConf().option(SparkReadOptions.TAG).parseOptional(); - } - public boolean streamingSkipDeleteSnapshots() { return confParser .booleanConf() @@ -356,4 +342,27 @@ public ParquetReaderType parquetReaderType() { .defaultValue(SparkSQLProperties.PARQUET_READER_TYPE_DEFAULT) .parse(); } + + public Pair incrementalAppendScanBoundaries() { + Long startSnapshotId = startSnapshotId(); + Long endSnapshotId = endSnapshotId(); + Long startTimestamp = startTimestamp(); + Long endTimestamp = endTimestamp(); + + Preconditions.checkArgument( + startTimestamp == null && endTimestamp == null, + "Only changelog scans support `%s` and `%s`. Use `%s` and `%s` for incremental scans.", + SparkReadOptions.START_TIMESTAMP, + SparkReadOptions.END_TIMESTAMP, + SparkReadOptions.START_SNAPSHOT_ID, + SparkReadOptions.END_SNAPSHOT_ID); + + Preconditions.checkArgument( + startSnapshotId != null, + "Cannot set only `%s` for incremental scans. Please, set `%s` too.", + SparkReadOptions.END_SNAPSHOT_ID, + SparkReadOptions.START_SNAPSHOT_ID); + + return Pair.of(startSnapshotId, endSnapshotId); + } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java index 8071b1db5b92..829fb9cdd2e1 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java @@ -23,8 +23,14 @@ public class SparkReadOptions { private SparkReadOptions() {} - // Snapshot ID of the table snapshot to read - public static final String SNAPSHOT_ID = "snapshot-id"; + // legacy time travel option that is no longer supported + public static final String LEGACY_SNAPSHOT_ID = "snapshot-id"; + + // legacy time travel option that is no longer supported + public static final String LEGACY_AS_OF_TIMESTAMP = "as-of-timestamp"; + + // legacy time travel option that is no longer supported + public static final String LEGACY_TAG = "tag"; // Start snapshot ID used in incremental scans (exclusive) public static final String START_SNAPSHOT_ID = "start-snapshot-id"; @@ -38,15 +44,9 @@ private SparkReadOptions() {} // End timestamp used in multi-snapshot scans (inclusive) public static final String END_TIMESTAMP = "end-timestamp"; - // A timestamp in milliseconds; the snapshot used will be the snapshot current at this time. - public static final String AS_OF_TIMESTAMP = "as-of-timestamp"; - // Branch to read from public static final String BRANCH = "branch"; - // Tag to read from - public static final String TAG = "tag"; - // Overrides the table's read.split.target-size and read.split.metadata-target-size public static final String SPLIT_SIZE = "split-size"; diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java index f49660a9f27c..942134eed623 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java @@ -154,7 +154,7 @@ public Table loadTable(Identifier ident) throws NoSuchTableException { public Table loadTable(Identifier ident, String version) throws NoSuchTableException { try { return icebergCatalog.loadTable(ident, version); - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + } catch (NoSuchTableException e) { return getSessionCatalog().loadTable(ident, version); } } @@ -163,7 +163,7 @@ public Table loadTable(Identifier ident, String version) throws NoSuchTableExcep public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableException { try { return icebergCatalog.loadTable(ident, timestamp); - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + } catch (NoSuchTableException e) { return getSessionCatalog().loadTable(ident, timestamp); } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index 593154d72a9c..609a088cbf33 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -987,7 +987,8 @@ private static void deleteManifests(FileIO io, List manifests) { } public static Dataset loadTable(SparkSession spark, Table table, long snapshotId) { - SparkTable sparkTable = new SparkTable(table, snapshotId, false); + TimeTravel timeTravel = TimeTravel.version(String.valueOf(snapshotId)); + SparkTable sparkTable = SparkTable.create(table, timeTravel); DataSourceV2Relation relation = createRelation(sparkTable, ImmutableMap.of()); Preconditions.checkArgument( spark instanceof org.apache.spark.sql.classic.SparkSession, @@ -1006,7 +1007,7 @@ public static Dataset loadMetadataTable( public static Dataset loadMetadataTable( SparkSession spark, Table table, MetadataTableType type, Map extraOptions) { Table metadataTable = MetadataTableUtils.createMetadataTableInstance(table, type); - SparkTable sparkMetadataTable = new SparkTable(metadataTable, false); + SparkTable sparkMetadataTable = new SparkTable(metadataTable); DataSourceV2Relation relation = createRelation(sparkMetadataTable, extraOptions); Preconditions.checkArgument( spark instanceof org.apache.spark.sql.classic.SparkSession, @@ -1024,8 +1025,26 @@ private static DataSourceV2Relation createRelation( sparkTable, Option.empty(), Option.empty(), options, Option.empty()); } - public static String determineWriteBranch(SparkSession spark, Table table, String branch) { - return determineWriteBranch(spark, table, branch, CaseInsensitiveStringMap.empty()); + public static void validateWriteBranch( + SparkSession spark, Table table, String branch, CaseInsensitiveStringMap options) { + validateBranch(spark, branch, determineWriteBranch(spark, table, branch, options)); + } + + public static void validateReadBranch( + SparkSession spark, Table table, String branch, CaseInsensitiveStringMap options) { + validateBranch(spark, branch, determineReadBranch(spark, table, branch, options)); + } + + private static void validateBranch(SparkSession spark, String branch, String targetBranch) { + Preconditions.checkArgument( + Objects.equal(branch, targetBranch) || Spark3Util.extensionsEnabled(spark), + "Must enable Iceberg extensions to use branching via options or SQL: operation targets branch `%s`", + targetBranch); + } + + public static String determineWriteBranch( + SparkSession spark, SparkTable sparkTable, CaseInsensitiveStringMap options) { + return determineWriteBranch(spark, sparkTable.table(), sparkTable.branch(), options); } /** @@ -1069,6 +1088,11 @@ public static String determineWriteBranch( return branch; } + public static String determineReadBranch( + SparkSession spark, SparkTable sparkTable, CaseInsensitiveStringMap options) { + return determineReadBranch(spark, sparkTable.table(), sparkTable.branch(), options); + } + /** * Determine the read branch. * @@ -1122,7 +1146,7 @@ private static String wapSessionBranch(SparkSession spark) { return wapBranch; } - public static boolean wapEnabled(Table table) { + private static boolean wapEnabled(Table table) { return PropertyUtil.propertyAsBoolean( table.properties(), TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java index d74754e4d3ce..5ad2545ca614 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java @@ -86,7 +86,6 @@ public class SparkWriteConf { private final SparkSession spark; private final Table table; - private final String branch; private final RuntimeConfig sessionConf; private final CaseInsensitiveStringMap options; private final SparkConfParser confParser; @@ -103,7 +102,6 @@ public SparkWriteConf( SparkSession spark, Table table, String branch, CaseInsensitiveStringMap options) { this.spark = spark; this.table = table; - this.branch = branch; this.sessionConf = spark.conf(); this.options = options; this.confParser = new SparkConfParser(spark, table, options); @@ -449,9 +447,10 @@ public Long validateFromSnapshotId() { } public IsolationLevel isolationLevel() { - String isolationLevelName = - confParser.stringConf().option(SparkWriteOptions.ISOLATION_LEVEL).parseOptional(); - return isolationLevelName != null ? IsolationLevel.fromName(isolationLevelName) : null; + return confParser + .enumConf(IsolationLevel::fromName) + .option(SparkWriteOptions.ISOLATION_LEVEL) + .parseOptional(); } public boolean caseSensitive() { @@ -462,10 +461,6 @@ public boolean caseSensitive() { .parse(); } - public String branch() { - return SparkTableUtil.determineWriteBranch(spark, table, branch, options); - } - public Map writeProperties() { Map writeProperties = Maps.newHashMap(); writeProperties.putAll(dataWriteProperties()); diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/TimeTravel.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/TimeTravel.java new file mode 100644 index 000000000000..2926aee8cc9f --- /dev/null +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/TimeTravel.java @@ -0,0 +1,64 @@ +/* + * 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; + +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.relocated.com.google.common.primitives.Longs; + +/** Represents a time-travel specification for loading tables at a specific point in time. */ +public sealed interface TimeTravel permits TimeTravel.AsOfVersion, TimeTravel.AsOfTimestamp { + + static TimeTravel version(String version) { + return new AsOfVersion(version); + } + + static TimeTravel timestampMicros(long timestamp) { + return new AsOfTimestamp(timestamp); + } + + static TimeTravel timestampMillis(long timestamp) { + return new AsOfTimestamp(TimeUnit.MILLISECONDS.toMicros(timestamp)); + } + + /** Time-travel specification using a version (snapshot ID, branch, or tag). */ + record AsOfVersion(String version) implements TimeTravel { + + public boolean isSnapshotId() { + return Longs.tryParse(version) != null; + } + + @Override + public String toString() { + return "VERSION AS OF '" + version + "'"; + } + } + + /** Time-travel specification using a timestamp in microseconds. */ + record AsOfTimestamp(long timestampMicros) implements TimeTravel { + + public long timestampMillis() { + return TimeUnit.MICROSECONDS.toMillis(timestampMicros); + } + + @Override + public String toString() { + return "TIMESTAMP AS OF " + timestampMicros; + } + } +} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseSparkScanBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseSparkScanBuilder.java new file mode 100644 index 000000000000..ec72a1505a66 --- /dev/null +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseSparkScanBuilder.java @@ -0,0 +1,261 @@ +/* + * 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 static org.apache.iceberg.TableProperties.SPLIT_LOOKBACK; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.iceberg.TableProperties.SPLIT_SIZE; + +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Scan; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionUtil; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.metrics.InMemoryMetricsReporter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkV2Filters; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.TypeUtil.GetID; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.expressions.filter.Predicate; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A base Spark scan builder with common functionality like projection and predicate pushdown. + * + *

Note that this class intentionally doesn't implement any optional mix-in Spark interfaces even + * if it contains necessary logic, allowing each concrete scan implementation to select what + * functionality is applicable to that scan. + */ +abstract class BaseSparkScanBuilder implements ScanBuilder { + + private static final Logger LOG = LoggerFactory.getLogger(BaseSparkScanBuilder.class); + private static final Predicate[] NO_PREDICATES = new Predicate[0]; + + private final SparkSession spark; + private final Table table; + private final Schema schema; + private final SparkReadConf readConf; + private final boolean caseSensitive; + private final Set metaFieldNames = Sets.newLinkedHashSet(); + private final InMemoryMetricsReporter metricsReporter = new InMemoryMetricsReporter(); + + private Schema projection; + private List filters = Lists.newArrayList(); + private Predicate[] pushedPredicates = NO_PREDICATES; + private Integer limit = null; + + protected BaseSparkScanBuilder( + SparkSession spark, Table table, Schema schema, CaseInsensitiveStringMap options) { + this.spark = spark; + this.table = table; + this.schema = schema; + this.readConf = new SparkReadConf(spark, table, options); + this.caseSensitive = readConf.caseSensitive(); + this.projection = schema; + } + + protected SparkSession spark() { + return spark; + } + + protected Table table() { + return table; + } + + protected Schema schema() { + return schema; + } + + protected Schema projection() { + return projection; + } + + protected SparkReadConf readConf() { + return readConf; + } + + protected boolean caseSensitive() { + return caseSensitive; + } + + protected List filters() { + return filters; + } + + protected Expression filter() { + return filters.stream().reduce(Expressions.alwaysTrue(), Expressions::and); + } + + protected InMemoryMetricsReporter metricsReporter() { + return metricsReporter; + } + + // logic necessary for SupportsPushDownRequiredColumns + public void pruneColumns(StructType requestedType) { + List dataFields = Lists.newArrayList(); + + for (StructField field : requestedType.fields()) { + if (MetadataColumns.isMetadataColumn(field.name())) { + metaFieldNames.add(field.name()); + } else { + dataFields.add(field); + } + } + + StructType requestedDataType = SparkSchemaUtil.toStructType(dataFields); + this.projection = SparkSchemaUtil.prune(projection, requestedDataType, filter(), caseSensitive); + } + + // logic necessary for SupportsPushDownV2Filters + public Predicate[] pushPredicates(Predicate[] predicates) { + // there are 3 kinds of filters: + // (1) filters that can be pushed down completely and don't have to evaluated by Spark + // (e.g. filters that select entire partitions) + // (2) filters that can be pushed down partially and require record-level filtering in Spark + // (e.g. filters that may select some but not necessarily all rows in a file) + // (3) filters that can't be pushed down at all and have to be evaluated by Spark + // (e.g. unsupported filters) + // filters (1) and (2) are used to prune files during job planning in Iceberg + // filters (2) and (3) form a set of post scan filters and must be evaluated by Spark + + List expressions = Lists.newArrayListWithExpectedSize(predicates.length); + List pushablePredicates = Lists.newArrayListWithExpectedSize(predicates.length); + List postScanPredicates = Lists.newArrayListWithExpectedSize(predicates.length); + + for (Predicate predicate : predicates) { + try { + Expression expr = SparkV2Filters.convert(predicate); + + if (expr != null) { + // try binding the expression to ensure it can be pushed down + Binder.bind(projection.asStruct(), expr, caseSensitive); + expressions.add(expr); + pushablePredicates.add(predicate); + } + + if (expr == null || !ExpressionUtil.selectsPartitions(expr, table, caseSensitive)) { + postScanPredicates.add(predicate); + } else { + LOG.info("Evaluating completely on Iceberg side: {}", predicate); + } + + } catch (Exception e) { + LOG.warn("Failed to check if {} can be pushed down: {}", predicate, e.getMessage()); + postScanPredicates.add(predicate); + } + } + + this.filters = expressions; + this.pushedPredicates = pushablePredicates.toArray(new Predicate[0]); + + return postScanPredicates.toArray(new Predicate[0]); + } + + // logic necessary for SupportsPushDownV2Filters + public Predicate[] pushedPredicates() { + return pushedPredicates; + } + + // logic necessary for SupportsPushDownLimit + public boolean pushLimit(int newLimit) { + this.limit = newLimit; + return true; + } + + // schema of rows that must be returned by readers + protected Schema projectionWithMetadataColumns() { + return TypeUtil.join(projection, calculateMetadataSchema()); + } + + // computes metadata schema avoiding conflicts between partition and data field IDs + private Schema calculateMetadataSchema() { + List metaFields = metaFields(); + Optional partitionField = findPartitionField(metaFields); + + if (partitionField.isEmpty()) { + return new Schema(metaFields); + } + + Types.StructType partitionType = partitionField.get().type().asStructType(); + Set partitionFieldIds = TypeUtil.getProjectedIds(partitionType); + GetID getId = TypeUtil.reassignConflictingIds(partitionFieldIds, allUsedFieldIds()); + return new Schema(metaFields, getId); + } + + private List metaFields() { + return metaFieldNames.stream() + .map(name -> MetadataColumns.metadataColumn(table, name)) + .collect(Collectors.toList()); + } + + private Optional findPartitionField(List fields) { + return fields.stream() + .filter(field -> MetadataColumns.PARTITION_COLUMN_ID == field.fieldId()) + .findFirst(); + } + + // collects used data field IDs across all known table schemas + private Set allUsedFieldIds() { + return table.schemas().values().stream() + .flatMap(tableSchema -> TypeUtil.getProjectedIds(tableSchema.asStruct()).stream()) + .collect(Collectors.toSet()); + } + + protected > T configureSplitPlanning(T scan) { + T newScan = scan; + + Long splitSize = readConf.splitSizeOption(); + if (splitSize != null) { + newScan = newScan.option(SPLIT_SIZE, String.valueOf(splitSize)); + } + + Integer splitLookback = readConf.splitLookbackOption(); + if (splitLookback != null) { + newScan = newScan.option(SPLIT_LOOKBACK, String.valueOf(splitLookback)); + } + + Long splitOpenFileCost = readConf.splitOpenFileCostOption(); + if (splitOpenFileCost != null) { + newScan = newScan.option(SPLIT_OPEN_FILE_COST, String.valueOf(splitOpenFileCost)); + } + + if (limit != null) { + newScan = newScan.minRowsRequested(limit.longValue()); + } + + return newScan; + } +} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index 81aeb3a4aea2..9a4ab30fec15 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -53,7 +53,7 @@ class BatchDataReader extends BaseBatchReader this( partition.table(), partition.taskGroup(), - partition.expectedSchema(), + partition.projection(), partition.isCaseSensitive(), parquetBatchReadConf, orcBatchReadConf, diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index 5054e28ac116..417440d4b48a 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -51,7 +51,7 @@ class ChangelogRowReader extends BaseRowReader this( partition.table(), partition.taskGroup(), - partition.expectedSchema(), + partition.projection(), partition.isCaseSensitive(), partition.cacheDeleteFilesOnExecutors()); } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java index a0462e8f8982..fe098ee0bd1a 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java @@ -20,19 +20,17 @@ import java.util.Arrays; import java.util.Map; -import java.util.Objects; import java.util.Optional; -import java.util.stream.Stream; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.spark.PathIdentifier; import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.iceberg.spark.SparkCatalog; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkRewriteTableCatalog; import org.apache.iceberg.spark.SparkSessionCatalog; import org.apache.iceberg.spark.SparkTableCache; -import org.apache.iceberg.util.PropertyUtil; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.CatalogManager; @@ -48,19 +46,21 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** - * The IcebergSource loads/writes tables with format "iceberg". It can load paths and tables. + * Data source for reading and writing Iceberg tables using the "iceberg" format. * - *

How paths/tables are loaded when using spark.read().format("iceberg").load(table) + *

The path parameter is resolved in the following priority order: * - *

table = "file:///path/to/table" -> loads a HadoopTable at given path table = "tablename" - * -> loads currentCatalog.currentNamespace.tablename table = "catalog.tablename" -> load - * "tablename" from the specified catalog. table = "namespace.tablename" -> load - * "namespace.tablename" from current catalog table = "catalog.namespace.tablename" -> - * "namespace.tablename" from the specified catalog. table = "namespace1.namespace2.tablename" -> - * load "namespace1.namespace2.tablename" from current catalog - * - *

The above list is in order of priority. For example: a matching catalog will take priority - * over any namespace resolution. + *

    + *
  1. Rewrite catalog - If path is in the table cache, loads from the rewrite catalog + *
  2. File path - If path contains "/", loads a table at the specified path + *
  3. Catalog identifier - Otherwise resolves as: + *
      + *
    • {@code "tbl"} - {@code currentCatalog.currentNamespace.tbl} + *
    • {@code "cat.tbl"} - {@code tbl} from the specified catalog + *
    • {@code "ns.tbl"} - {@code ns.tbl} from the current catalog + *
    • {@code "cat.ns.tbl"} - {@code ns.tbl} from the specified catalog + *
    + *
*/ public class IcebergSource implements DataSourceRegister, SupportsCatalogOptions, SessionConfigSupport { @@ -69,10 +69,7 @@ public class IcebergSource private static final String CATALOG_PREFIX = "spark.sql.catalog."; private static final String DEFAULT_CATALOG = CATALOG_PREFIX + DEFAULT_CATALOG_NAME; private static final String REWRITE_CATALOG = CATALOG_PREFIX + REWRITE_CATALOG_NAME; - private static final String AT_TIMESTAMP = "at_timestamp_"; - private static final String SNAPSHOT_ID = "snapshot_id_"; private static final String BRANCH_PREFIX = "branch_"; - private static final String TAG_PREFIX = "tag_"; private static final String[] EMPTY_NAMESPACE = new String[0]; private static final SparkTableCache TABLE_CACHE = SparkTableCache.get(); @@ -104,96 +101,74 @@ public boolean supportsExternalMetadata() { @Override public Table getTable(StructType schema, Transform[] partitioning, Map options) { - Spark3Util.CatalogAndIdentifier catalogIdentifier = - catalogAndIdentifier(new CaseInsensitiveStringMap(options)); - CatalogPlugin catalog = catalogIdentifier.catalog(); - Identifier ident = catalogIdentifier.identifier(); + return loadTable(new CaseInsensitiveStringMap(options)); + } + private Table loadTable(CaseInsensitiveStringMap options) { + CatalogAndIdentifier catalogAndIdent = catalogAndIdentifier(options); + TableCatalog catalog = catalogAndIdent.tableCatalog(); + Identifier ident = catalogAndIdent.identifier(); try { - if (catalog instanceof TableCatalog) { - return ((TableCatalog) catalog).loadTable(ident); - } + return catalog.loadTable(ident); } catch (NoSuchTableException e) { - // throwing an iceberg NoSuchTableException because the Spark one is typed and can't be thrown - // from this interface + // TableProvider doesn't permit typed exception while loading tables, + // so throw Iceberg NoSuchTableException because the Spark one is typed throw new org.apache.iceberg.exceptions.NoSuchTableException( - e, "Cannot find table for %s.", ident); + e, + "Cannot find table %s in catalog %s (%s)", + ident, + catalog.name(), + catalog.getClass().getName()); } - - // throwing an iceberg NoSuchTableException because the Spark one is typed and can't be thrown - // from this interface - throw new org.apache.iceberg.exceptions.NoSuchTableException( - "Cannot find table for %s.", ident); } - private Spark3Util.CatalogAndIdentifier catalogAndIdentifier(CaseInsensitiveStringMap options) { + private CatalogAndIdentifier catalogAndIdentifier(CaseInsensitiveStringMap options) { Preconditions.checkArgument( options.containsKey(SparkReadOptions.PATH), "Cannot open table: path is not set"); + Spark3Util.validateNoLegacyTimeTravel(options); + SparkSession spark = SparkSession.active(); + CatalogManager catalogManager = spark.sessionState().catalogManager(); + setupDefaultSparkCatalogs(spark); - String path = options.get(SparkReadOptions.PATH); - Long snapshotId = propertyAsLong(options, SparkReadOptions.SNAPSHOT_ID); - Long asOfTimestamp = propertyAsLong(options, SparkReadOptions.AS_OF_TIMESTAMP); + String path = options.get(SparkReadOptions.PATH); String branch = options.get(SparkReadOptions.BRANCH); - String tag = options.get(SparkReadOptions.TAG); - Preconditions.checkArgument( - Stream.of(snapshotId, asOfTimestamp, branch, tag).filter(Objects::nonNull).count() <= 1, - "Can specify only one of snapshot-id (%s), as-of-timestamp (%s), branch (%s), tag (%s)", - snapshotId, - asOfTimestamp, - branch, - tag); - - String selector = null; - - if (snapshotId != null) { - selector = SNAPSHOT_ID + snapshotId; - } - - if (asOfTimestamp != null) { - selector = AT_TIMESTAMP + asOfTimestamp; - } - - if (branch != null) { - selector = BRANCH_PREFIX + branch; - } - - if (tag != null) { - selector = TAG_PREFIX + tag; - } - - CatalogManager catalogManager = spark.sessionState().catalogManager(); + String branchSelector = branch != null ? BRANCH_PREFIX + branch : null; // return rewrite catalog with path as group ID if table is staged for rewrite if (TABLE_CACHE.contains(path)) { - return new Spark3Util.CatalogAndIdentifier( + return new CatalogAndIdentifier( catalogManager.catalog(REWRITE_CATALOG_NAME), Identifier.of(EMPTY_NAMESPACE, path)); } + // return default catalog and PathIdentifier with branch selector for a path if (path.contains("/")) { - // contains a path. Return iceberg default catalog and a PathIdentifier - return new Spark3Util.CatalogAndIdentifier( + return new CatalogAndIdentifier( catalogManager.catalog(DEFAULT_CATALOG_NAME), - new PathIdentifier(pathWithSelector(path, selector))); + new PathIdentifier(pathWithSelector(path, branchSelector))); } - final Spark3Util.CatalogAndIdentifier catalogAndIdentifier = - Spark3Util.catalogAndIdentifier("path or identifier", spark, path); + // treat path as an identifier and resolve it against the session config + // if the catalog resolves to an unknown session catalog, use the default Iceberg catalog + CatalogAndIdentifier catalogAndIdent = resolveIdentifier(spark, path); + CatalogPlugin catalog = catalogAndIdent.catalog(); + Identifier ident = catalogAndIdent.identifier(); + return new CatalogAndIdentifier( + isUnknownSessionCatalog(catalog) ? catalogManager.catalog(DEFAULT_CATALOG_NAME) : catalog, + identifierWithSelector(ident, branchSelector)); + } - Identifier ident = identifierWithSelector(catalogAndIdentifier.identifier(), selector); - if (catalogAndIdentifier.catalog().name().equals("spark_catalog") - && !(catalogAndIdentifier.catalog() instanceof SparkSessionCatalog)) { - // catalog is a session catalog but does not support Iceberg. Use Iceberg instead. - return new Spark3Util.CatalogAndIdentifier( - catalogManager.catalog(DEFAULT_CATALOG_NAME), ident); - } else { - return new Spark3Util.CatalogAndIdentifier(catalogAndIdentifier.catalog(), ident); - } + private static CatalogAndIdentifier resolveIdentifier(SparkSession spark, String ident) { + return Spark3Util.catalogAndIdentifier("identifier", spark, ident); + } + + private static boolean isUnknownSessionCatalog(CatalogPlugin catalog) { + return catalog.name().equals("spark_catalog") && !(catalog instanceof SparkSessionCatalog); } private String pathWithSelector(String path, String selector) { - return (selector == null) ? path : path + "#" + selector; + return selector == null ? path : path + "#" + selector; } private Identifier identifierWithSelector(Identifier ident, String selector) { @@ -219,23 +194,12 @@ public String extractCatalog(CaseInsensitiveStringMap options) { @Override public Optional extractTimeTravelVersion(CaseInsensitiveStringMap options) { - return Optional.ofNullable( - PropertyUtil.propertyAsString(options, SparkReadOptions.VERSION_AS_OF, null)); + return Optional.ofNullable(options.get(SparkReadOptions.VERSION_AS_OF)); } @Override public Optional extractTimeTravelTimestamp(CaseInsensitiveStringMap options) { - return Optional.ofNullable( - PropertyUtil.propertyAsString(options, SparkReadOptions.TIMESTAMP_AS_OF, null)); - } - - private static Long propertyAsLong(CaseInsensitiveStringMap options, String property) { - String value = options.get(property); - if (value != null) { - return Long.parseLong(value); - } - - return null; + return Optional.ofNullable(options.get(SparkReadOptions.TIMESTAMP_AS_OF)); } private static void setupDefaultSparkCatalogs(SparkSession spark) { diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java index c93e9e79835b..1a45facba6c6 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java @@ -49,7 +49,7 @@ class PositionDeletesRowReader extends BaseRowReader this( partition.table(), partition.taskGroup(), - partition.expectedSchema(), + partition.projection(), partition.isCaseSensitive(), partition.cacheDeleteFilesOnExecutors()); } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index a4f3ecd8e336..0b53e72d9908 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -48,7 +48,7 @@ class RowDataReader extends BaseRowReader implements PartitionRead this( partition.table(), partition.taskGroup(), - partition.expectedSchema(), + partition.projection(), partition.isCaseSensitive(), partition.cacheDeleteFilesOnExecutors()); } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index d05d9a16570e..a4d143fe9321 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -50,7 +50,7 @@ class SparkBatch implements Batch { private final SparkReadConf readConf; private final Types.StructType groupingKeyType; private final List> taskGroups; - private final Schema expectedSchema; + private final Schema projection; private final boolean caseSensitive; private final boolean localityEnabled; private final boolean executorCacheLocalityEnabled; @@ -63,14 +63,14 @@ class SparkBatch implements Batch { SparkReadConf readConf, Types.StructType groupingKeyType, List> taskGroups, - Schema expectedSchema, + Schema projection, int scanHashCode) { this.sparkContext = sparkContext; this.table = table; this.readConf = readConf; this.groupingKeyType = groupingKeyType; this.taskGroups = taskGroups; - this.expectedSchema = expectedSchema; + this.projection = projection; this.caseSensitive = readConf.caseSensitive(); this.localityEnabled = readConf.localityEnabled(); this.executorCacheLocalityEnabled = readConf.executorCacheLocalityEnabled(); @@ -83,7 +83,7 @@ public InputPartition[] planInputPartitions() { // broadcast the table metadata as input partitions will be sent to executors Broadcast tableBroadcast = sparkContext.broadcast(SerializableTableWithSize.copyOf(table)); - String expectedSchemaString = SchemaParser.toJson(expectedSchema); + String projectionString = SchemaParser.toJson(projection); String[][] locations = computePreferredLocations(); InputPartition[] partitions = new InputPartition[taskGroups.size()]; @@ -94,7 +94,7 @@ public InputPartition[] planInputPartitions() { groupingKeyType, taskGroups.get(index), tableBroadcast, - expectedSchemaString, + projectionString, caseSensitive, locations != null ? locations[index] : SparkPlanningUtil.NO_LOCATION_PREFERENCE, cacheDeleteFilesOnExecutors); @@ -150,7 +150,7 @@ private OrcBatchReadConf orcBatchReadConf() { // - all tasks are of FileScanTask type and read only Parquet files private boolean useParquetBatchReads() { return readConf.parquetVectorizationEnabled() - && expectedSchema.columns().stream().allMatch(this::supportsParquetBatchReads) + && projection.columns().stream().allMatch(this::supportsParquetBatchReads) && taskGroups.stream().allMatch(this::supportsParquetBatchReads); } @@ -175,7 +175,7 @@ private boolean supportsParquetBatchReads(Types.NestedField field) { private boolean useCometBatchReads() { return readConf.parquetVectorizationEnabled() && readConf.parquetReaderType() == ParquetReaderType.COMET - && expectedSchema.columns().stream().allMatch(this::supportsCometBatchReads) + && projection.columns().stream().allMatch(this::supportsCometBatchReads) && taskGroups.stream().allMatch(this::supportsParquetBatchReads); } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java index 0ec77d9d0637..814ca410d147 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -19,229 +19,51 @@ package org.apache.iceberg.spark.source; import java.util.List; -import java.util.Map; import java.util.Objects; -import java.util.Set; import java.util.function.Supplier; -import java.util.stream.Collectors; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileContent; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionScanTask; -import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Scan; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.expressions.Binder; -import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.ExpressionUtil; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.expressions.Projections; import org.apache.iceberg.metrics.ScanReport; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.SparkV2Filters; -import org.apache.iceberg.util.ContentFileUtil; -import org.apache.iceberg.util.DeleteFileSet; -import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.connector.expressions.NamedReference; -import org.apache.spark.sql.connector.expressions.filter.Predicate; import org.apache.spark.sql.connector.read.Statistics; -import org.apache.spark.sql.connector.read.SupportsRuntimeV2Filtering; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -class SparkBatchQueryScan extends SparkPartitioningAwareScan - implements SupportsRuntimeV2Filtering { +class SparkBatchQueryScan extends SparkRuntimeFilterableScan { - private static final Logger LOG = LoggerFactory.getLogger(SparkBatchQueryScan.class); - - private final Long snapshotId; - private final Long startSnapshotId; - private final Long endSnapshotId; - private final Long asOfTimestamp; - private final String tag; - private final List runtimeFilterExpressions; + private final Snapshot snapshot; + private final String branch; SparkBatchQueryScan( SparkSession spark, Table table, + Schema schema, + Snapshot snapshot, + String branch, Scan> scan, SparkReadConf readConf, - Schema expectedSchema, + Schema projection, List filters, Supplier scanReportSupplier) { - super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier); - - this.snapshotId = readConf.snapshotId(); - this.startSnapshotId = readConf.startSnapshotId(); - this.endSnapshotId = readConf.endSnapshotId(); - this.asOfTimestamp = readConf.asOfTimestamp(); - this.tag = readConf.tag(); - this.runtimeFilterExpressions = Lists.newArrayList(); + super(spark, table, schema, scan, readConf, projection, filters, scanReportSupplier); + this.snapshot = snapshot; + this.branch = branch; } Long snapshotId() { - return snapshotId; - } - - @Override - protected Class taskJavaClass() { - return PartitionScanTask.class; - } - - @Override - public NamedReference[] filterAttributes() { - Set partitionFieldSourceIds = Sets.newHashSet(); - - for (PartitionSpec spec : specs()) { - for (PartitionField field : spec.fields()) { - partitionFieldSourceIds.add(field.sourceId()); - } - } - - Map quotedNameById = SparkSchemaUtil.indexQuotedNameById(expectedSchema()); - - // the optimizer will look for an equality condition with filter attributes in a join - // as the scan has been already planned, filtering can only be done on projected attributes - // that's why only partition source fields that are part of the read schema can be reported - - return partitionFieldSourceIds.stream() - .filter(fieldId -> expectedSchema().findField(fieldId) != null) - .map(fieldId -> Spark3Util.toNamedReference(quotedNameById.get(fieldId))) - .toArray(NamedReference[]::new); - } - - @Override - public void filter(Predicate[] predicates) { - Expression runtimeFilterExpr = convertRuntimeFilters(predicates); - - if (runtimeFilterExpr != Expressions.alwaysTrue()) { - Map evaluatorsBySpecId = Maps.newHashMap(); - - for (PartitionSpec spec : specs()) { - Expression inclusiveExpr = - Projections.inclusive(spec, caseSensitive()).project(runtimeFilterExpr); - Evaluator inclusive = new Evaluator(spec.partitionType(), inclusiveExpr); - evaluatorsBySpecId.put(spec.specId(), inclusive); - } - - List filteredTasks = - tasks().stream() - .filter( - task -> { - Evaluator evaluator = evaluatorsBySpecId.get(task.spec().specId()); - return evaluator.eval(task.partition()); - }) - .collect(Collectors.toList()); - - LOG.info( - "{} of {} task(s) for table {} matched runtime filter {}", - filteredTasks.size(), - tasks().size(), - table().name(), - ExpressionUtil.toSanitizedString(runtimeFilterExpr)); - - // don't invalidate tasks if the runtime filter had no effect to avoid planning splits again - if (filteredTasks.size() < tasks().size()) { - resetTasks(filteredTasks); - } - - // save the evaluated filter for equals/hashCode - runtimeFilterExpressions.add(runtimeFilterExpr); - } - } - - protected Map rewritableDeletes(boolean forDVs) { - Map rewritableDeletes = Maps.newHashMap(); - - for (ScanTask task : tasks()) { - FileScanTask fileScanTask = task.asFileScanTask(); - for (DeleteFile deleteFile : fileScanTask.deletes()) { - if (shouldRewrite(deleteFile, forDVs)) { - rewritableDeletes - .computeIfAbsent(fileScanTask.file().location(), ignored -> DeleteFileSet.create()) - .add(deleteFile); - } - } - } - - return rewritableDeletes; - } - - // for DVs all position deletes must be rewritten - // for position deletes, only file-scoped deletes must be rewritten - private boolean shouldRewrite(DeleteFile deleteFile, boolean forDVs) { - if (forDVs) { - return deleteFile.content() != FileContent.EQUALITY_DELETES; - } - - return ContentFileUtil.isFileScoped(deleteFile); - } - - // at this moment, Spark can only pass IN filters for a single attribute - // if there are multiple filter attributes, Spark will pass two separate IN filters - private Expression convertRuntimeFilters(Predicate[] predicates) { - Expression runtimeFilterExpr = Expressions.alwaysTrue(); - - for (Predicate predicate : predicates) { - Expression expr = SparkV2Filters.convert(predicate); - if (expr != null) { - try { - Binder.bind(expectedSchema().asStruct(), expr, caseSensitive()); - runtimeFilterExpr = Expressions.and(runtimeFilterExpr, expr); - } catch (ValidationException e) { - LOG.warn("Failed to bind {} to expected schema, skipping runtime filter", expr, e); - } - } else { - LOG.warn("Unsupported runtime filter {}", predicate); - } - } - - return runtimeFilterExpr; + return snapshot != null ? snapshot.snapshotId() : null; } @Override public Statistics estimateStatistics() { - if (scan() == null) { - return estimateStatistics(null); - - } else if (snapshotId != null) { - Snapshot snapshot = table().snapshot(snapshotId); - return estimateStatistics(snapshot); - - } else if (asOfTimestamp != null) { - long snapshotIdAsOfTime = SnapshotUtil.snapshotIdAsOfTime(table(), asOfTimestamp); - Snapshot snapshot = table().snapshot(snapshotIdAsOfTime); - return estimateStatistics(snapshot); - - } else if (branch() != null) { - Snapshot snapshot = table().snapshot(branch()); - return estimateStatistics(snapshot); - - } else if (tag != null) { - Snapshot snapshot = table().snapshot(tag); - return estimateStatistics(snapshot); - - } else { - Snapshot snapshot = table().currentSnapshot(); - return estimateStatistics(snapshot); - } + return estimateStatistics(snapshot); } @Override - @SuppressWarnings("checkstyle:CyclomaticComplexity") public boolean equals(Object o) { if (this == o) { return true; @@ -253,40 +75,36 @@ public boolean equals(Object o) { SparkBatchQueryScan that = (SparkBatchQueryScan) o; return table().name().equals(that.table().name()) - && Objects.equals(branch(), that.branch()) + && Objects.equals(table().uuid(), that.table().uuid()) + && Objects.equals(snapshot, that.snapshot) + && Objects.equals(branch, that.branch) && readSchema().equals(that.readSchema()) // compare Spark schemas to ignore field ids && filtersDesc().equals(that.filtersDesc()) - && runtimeFiltersDesc().equals(that.runtimeFiltersDesc()) - && Objects.equals(snapshotId, that.snapshotId) - && Objects.equals(startSnapshotId, that.startSnapshotId) - && Objects.equals(endSnapshotId, that.endSnapshotId) - && Objects.equals(asOfTimestamp, that.asOfTimestamp) - && Objects.equals(tag, that.tag); + && runtimeFiltersDesc().equals(that.runtimeFiltersDesc()); } @Override public int hashCode() { return Objects.hash( table().name(), - branch(), + table().uuid(), + snapshot, + branch, readSchema(), filtersDesc(), - runtimeFiltersDesc(), - snapshotId, - startSnapshotId, - endSnapshotId, - asOfTimestamp, - tag); + runtimeFiltersDesc()); } @Override public String description() { return String.format( - "IcebergScan(table=%s, branch=%s, filters=%s, runtimeFilters=%s, groupedBy=%s)", - table(), branch(), filtersDesc(), runtimeFiltersDesc(), groupingKeyDesc()); - } - - private String runtimeFiltersDesc() { - return Spark3Util.describe(runtimeFilterExpressions); + "IcebergScan(table=%s, schemaId=%s, snapshotId=%s, branch=%s, filters=%s, runtimeFilters=%s, groupedBy=%s)", + table(), + schema().schemaId(), + snapshotId(), + branch, + filtersDesc(), + runtimeFiltersDesc(), + groupingKeyDesc()); } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java index eb4659f3ebc5..6a10b31efd6d 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java @@ -67,8 +67,7 @@ class SparkChangelogScan implements Scan, SupportsReportStatistics { IncrementalChangelogScan scan, SparkReadConf readConf, Schema expectedSchema, - List filters, - boolean emptyScan) { + List filters) { SparkSchemaUtil.validateMetadataColumnReferences(table.schema(), expectedSchema); @@ -80,7 +79,7 @@ class SparkChangelogScan implements Scan, SupportsReportStatistics { this.filters = filters != null ? filters : Collections.emptyList(); this.startSnapshotId = readConf.startSnapshotId(); this.endSnapshotId = readConf.endSnapshotId(); - if (emptyScan) { + if (scan == null) { this.taskGroups = Collections.emptyList(); } } @@ -148,6 +147,7 @@ public boolean equals(Object o) { SparkChangelogScan that = (SparkChangelogScan) o; return table.name().equals(that.table.name()) + && Objects.equals(table.uuid(), that.table.uuid()) && readSchema().equals(that.readSchema()) // compare Spark schemas to ignore field IDs && filtersDesc().equals(that.filtersDesc()) && Objects.equals(startSnapshotId, that.startSnapshotId) @@ -156,7 +156,8 @@ && filtersDesc().equals(that.filtersDesc()) @Override public int hashCode() { - return Objects.hash(table.name(), readSchema(), filtersDesc(), startSnapshotId, endSnapshotId); + return Objects.hash( + table.name(), table.uuid(), readSchema(), filtersDesc(), startSnapshotId, endSnapshotId); } private String filtersDesc() { diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScanBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScanBuilder.java new file mode 100644 index 000000000000..28bd2b523578 --- /dev/null +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScanBuilder.java @@ -0,0 +1,152 @@ +/* + * 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 org.apache.iceberg.IncrementalChangelogScan; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.SupportsPushDownLimit; +import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns; +import org.apache.spark.sql.connector.read.SupportsPushDownV2Filters; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +public class SparkChangelogScanBuilder extends BaseSparkScanBuilder + implements SupportsPushDownV2Filters, SupportsPushDownRequiredColumns, SupportsPushDownLimit { + + SparkChangelogScanBuilder( + SparkSession spark, Table table, Schema schema, CaseInsensitiveStringMap options) { + super(spark, table, schema, options); + } + + @Override + public Scan build() { + Long startSnapshotId = readConf().startSnapshotId(); + Long endSnapshotId = readConf().endSnapshotId(); + Long startTimestamp = readConf().startTimestamp(); + Long endTimestamp = readConf().endTimestamp(); + + Preconditions.checkArgument( + startSnapshotId == null || startTimestamp == null, + "Cannot set both %s and %s for changelogs", + SparkReadOptions.START_SNAPSHOT_ID, + SparkReadOptions.START_TIMESTAMP); + + Preconditions.checkArgument( + endSnapshotId == null || endTimestamp == null, + "Cannot set both %s and %s for changelogs", + SparkReadOptions.END_SNAPSHOT_ID, + SparkReadOptions.END_TIMESTAMP); + + Preconditions.checkArgument( + startTimestamp == null || endTimestamp == null || startTimestamp < endTimestamp, + "Cannot set %s to be greater than %s for changelogs", + SparkReadOptions.START_TIMESTAMP, + SparkReadOptions.END_TIMESTAMP); + + if (startTimestamp != null) { + if (noSnapshotsAfter(startTimestamp)) { + return emptyChangelogScan(); + } + startSnapshotId = getStartSnapshotId(startTimestamp); + } + + if (endTimestamp != null) { + endSnapshotId = getEndSnapshotId(endTimestamp); + if (noSnapshotsBetween(startSnapshotId, endSnapshotId)) { + return emptyChangelogScan(); + } + } + + Schema projection = projectionWithMetadataColumns(); + IncrementalChangelogScan scan = buildIcebergScan(projection, startSnapshotId, endSnapshotId); + return new SparkChangelogScan(spark(), table(), scan, readConf(), projection, filters()); + } + + private IncrementalChangelogScan buildIcebergScan( + Schema projection, Long startSnapshotId, Long endSnapshotId) { + IncrementalChangelogScan scan = + table() + .newIncrementalChangelogScan() + .caseSensitive(caseSensitive()) + .filter(filter()) + .project(projection) + .metricsReporter(metricsReporter()); + + if (startSnapshotId != null) { + scan = scan.fromSnapshotExclusive(startSnapshotId); + } + + if (endSnapshotId != null) { + scan = scan.toSnapshot(endSnapshotId); + } + + scan = configureSplitPlanning(scan); + + return scan; + } + + private SparkChangelogScan emptyChangelogScan() { + return new SparkChangelogScan( + spark(), + table(), + null /* no scan */, + readConf(), + projectionWithMetadataColumns(), + filters()); + } + + private boolean noSnapshotsAfter(long timestamp) { + Snapshot currentSnapshot = table().currentSnapshot(); + return currentSnapshot == null || timestamp > currentSnapshot.timestampMillis(); + } + + private boolean noSnapshotsBetween(Long startSnapshotId, Long endSnapshotId) { + return (startSnapshotId == null && endSnapshotId == null) + || (startSnapshotId != null && startSnapshotId.equals(endSnapshotId)); + } + + private Long getStartSnapshotId(Long startTimestamp) { + Snapshot oldestSnapshotAfter = SnapshotUtil.oldestAncestorAfter(table(), startTimestamp); + + if (oldestSnapshotAfter == null) { + return null; + } else if (oldestSnapshotAfter.timestampMillis() == startTimestamp) { + return oldestSnapshotAfter.snapshotId(); + } else { + return oldestSnapshotAfter.parentId(); + } + } + + private Long getEndSnapshotId(Long endTimestamp) { + Long endSnapshotId = null; + for (Snapshot snapshot : SnapshotUtil.currentAncestors(table())) { + if (snapshot.timestampMillis() <= endTimestamp) { + endSnapshotId = snapshot.snapshotId(); + break; + } + } + return endSnapshotId; + } +} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java index b9949085c74d..bdafca27fbb8 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java @@ -21,50 +21,49 @@ import java.util.Set; import org.apache.iceberg.ChangelogUtil; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.catalog.MetadataColumn; import org.apache.spark.sql.connector.catalog.SupportsMetadataColumns; import org.apache.spark.sql.connector.catalog.SupportsRead; -import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.connector.catalog.TableCapability; -import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -public class SparkChangelogTable implements Table, SupportsRead, SupportsMetadataColumns { +public class SparkChangelogTable + implements org.apache.spark.sql.connector.catalog.Table, SupportsRead, SupportsMetadataColumns { public static final String TABLE_NAME = "changes"; private static final Set CAPABILITIES = ImmutableSet.of(TableCapability.BATCH_READ); - private final org.apache.iceberg.Table icebergTable; - private final boolean refreshEagerly; + private final Table table; + private final Schema schema; private SparkSession lazySpark = null; - private StructType lazyTableSparkType = null; - private Schema lazyChangelogSchema = null; + private StructType lazySparkSchema = null; - public SparkChangelogTable(org.apache.iceberg.Table icebergTable, boolean refreshEagerly) { - this.icebergTable = icebergTable; - this.refreshEagerly = refreshEagerly; + public SparkChangelogTable(Table table) { + this.table = table; + this.schema = ChangelogUtil.changelogSchema(table.schema()); } @Override public String name() { - return icebergTable.name() + "." + TABLE_NAME; + return table.name() + "." + TABLE_NAME; } @Override public StructType schema() { - if (lazyTableSparkType == null) { - this.lazyTableSparkType = SparkSchemaUtil.convert(changelogSchema()); + if (lazySparkSchema == null) { + this.lazySparkSchema = SparkSchemaUtil.convert(schema); } - return lazyTableSparkType; + return lazySparkSchema; } @Override @@ -74,24 +73,7 @@ public Set capabilities() { @Override public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { - if (refreshEagerly) { - icebergTable.refresh(); - } - - return new SparkScanBuilder(spark(), icebergTable, changelogSchema(), options) { - @Override - public Scan build() { - return buildChangelogScan(); - } - }; - } - - private Schema changelogSchema() { - if (lazyChangelogSchema == null) { - this.lazyChangelogSchema = ChangelogUtil.changelogSchema(icebergTable.schema()); - } - - return lazyChangelogSchema; + return new SparkChangelogScanBuilder(spark(), table, schema, options); } private SparkSession spark() { @@ -106,7 +88,7 @@ private SparkSession spark() { public MetadataColumn[] metadataColumns() { return new MetadataColumn[] { SparkMetadataColumns.SPEC_ID, - SparkMetadataColumns.partition(icebergTable), + SparkMetadataColumns.partition(table), SparkMetadataColumns.FILE_PATH, SparkMetadataColumns.ROW_POSITION, SparkMetadataColumns.IS_DELETED, diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java index dd3d785b3b5f..75894598ddee 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableUtil; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -42,6 +43,7 @@ class SparkCopyOnWriteOperation implements RowLevelOperation { private final SparkSession spark; private final Table table; + private final Snapshot snapshot; private final String branch; private final Command command; private final IsolationLevel isolationLevel; @@ -54,11 +56,13 @@ class SparkCopyOnWriteOperation implements RowLevelOperation { SparkCopyOnWriteOperation( SparkSession spark, Table table, + Snapshot snapshot, String branch, RowLevelOperationInfo info, IsolationLevel isolationLevel) { this.spark = spark; this.table = table; + this.snapshot = snapshot; this.branch = branch; this.command = info.command(); this.isolationLevel = isolationLevel; @@ -73,7 +77,7 @@ public Command command() { public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { if (lazyScanBuilder == null) { lazyScanBuilder = - new SparkScanBuilder(spark, table, branch, options) { + new SparkScanBuilder(spark, table, table.schema(), snapshot, branch, options) { @Override public Scan build() { Scan scan = super.buildCopyOnWriteScan(); @@ -98,18 +102,18 @@ public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { @Override public NamedReference[] requiredMetadataAttributes() { - List metadataAttributes = Lists.newArrayList(); - metadataAttributes.add(Expressions.column(MetadataColumns.FILE_PATH.name())); + List metaAttrs = Lists.newArrayList(); + metaAttrs.add(Expressions.column(MetadataColumns.FILE_PATH.name())); + if (command == DELETE || command == UPDATE) { - metadataAttributes.add(Expressions.column(MetadataColumns.ROW_POSITION.name())); + metaAttrs.add(Expressions.column(MetadataColumns.ROW_POSITION.name())); } if (TableUtil.supportsRowLineage(table)) { - metadataAttributes.add(Expressions.column(MetadataColumns.ROW_ID.name())); - metadataAttributes.add( - Expressions.column(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name())); + metaAttrs.add(Expressions.column(MetadataColumns.ROW_ID.name())); + metaAttrs.add(Expressions.column(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name())); } - return metadataAttributes.toArray(NamedReference[]::new); + return metaAttrs.toArray(NamedReference[]::new); } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java index 38664ce8bbfd..ef09779aaede 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -32,10 +32,8 @@ import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.metrics.ScanReport; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkReadConf; -import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.expressions.Expressions; import org.apache.spark.sql.connector.expressions.NamedReference; @@ -52,31 +50,24 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan private static final Logger LOG = LoggerFactory.getLogger(SparkCopyOnWriteScan.class); private final Snapshot snapshot; - private Set filteredLocations = null; + private final String branch; - SparkCopyOnWriteScan( - SparkSession spark, - Table table, - SparkReadConf readConf, - Schema expectedSchema, - List filters, - Supplier scanReportSupplier) { - this(spark, table, null, null, readConf, expectedSchema, filters, scanReportSupplier); - } + private Set filteredLocations = null; SparkCopyOnWriteScan( SparkSession spark, Table table, - BatchScan scan, + Schema schema, Snapshot snapshot, + String branch, + BatchScan scan, SparkReadConf readConf, - Schema expectedSchema, + Schema projection, List filters, Supplier scanReportSupplier) { - super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier); - + super(spark, table, schema, scan, readConf, projection, filters, scanReportSupplier); this.snapshot = snapshot; - + this.branch = branch; if (scan == null) { this.filteredLocations = Collections.emptySet(); } @@ -104,15 +95,6 @@ public NamedReference[] filterAttributes() { @Override public void filter(Filter[] filters) { - Preconditions.checkState( - Objects.equals(snapshotId(), currentSnapshotId()), - "Runtime file filtering is not possible: the table has been concurrently modified. " - + "Row-level operation scan snapshot ID: %s, current table snapshot ID: %s. " - + "If an external process modifies the table, enable table caching in the catalog. " - + "If multiple threads modify the table, use independent Spark sessions in each thread.", - snapshotId(), - currentSnapshotId()); - for (Filter filter : filters) { // Spark can only pass In filters at the moment if (filter instanceof In @@ -161,27 +143,30 @@ public boolean equals(Object o) { SparkCopyOnWriteScan that = (SparkCopyOnWriteScan) o; return table().name().equals(that.table().name()) + && Objects.equals(table().uuid(), that.table().uuid()) + && Objects.equals(snapshot, that.snapshot) + && Objects.equals(branch, that.branch) && readSchema().equals(that.readSchema()) // compare Spark schemas to ignore field ids && filtersDesc().equals(that.filtersDesc()) - && Objects.equals(snapshotId(), that.snapshotId()) && Objects.equals(filteredLocations, that.filteredLocations); } @Override public int hashCode() { return Objects.hash( - table().name(), readSchema(), filtersDesc(), snapshotId(), filteredLocations); + table().name(), + table().uuid(), + snapshot, + branch, + readSchema(), + filtersDesc(), + filteredLocations); } @Override public String description() { return String.format( - "IcebergCopyOnWriteScan(table=%s, filters=%s, groupedBy=%s)", - table(), filtersDesc(), groupingKeyDesc()); - } - - private Long currentSnapshotId() { - Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table(), branch()); - return currentSnapshot != null ? currentSnapshot.snapshotId() : null; + "IcebergCopyOnWriteScan(table=%s, schemaId=%s, snapshotId=%s, branch=%s, filters=%s, groupedBy=%s)", + table(), schema().schemaId(), snapshotId(), branch, filtersDesc(), groupingKeyDesc()); } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkIncrementalScan.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkIncrementalScan.java new file mode 100644 index 000000000000..890d3f2ab05e --- /dev/null +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkIncrementalScan.java @@ -0,0 +1,97 @@ +/* + * 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.Objects; +import java.util.function.Supplier; +import org.apache.iceberg.Scan; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.spark.sql.SparkSession; + +class SparkIncrementalScan extends SparkRuntimeFilterableScan { + + private final long startSnapshotId; + private final Long endSnapshotId; + + SparkIncrementalScan( + SparkSession spark, + Table table, + long startSnapshotId, + Long endSnapshotId, + Scan> scan, + SparkReadConf readConf, + Schema projection, + List filters, + Supplier scanReportSupplier) { + super(spark, table, table.schema(), scan, readConf, projection, filters, scanReportSupplier); + this.startSnapshotId = startSnapshotId; + this.endSnapshotId = endSnapshotId; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + SparkIncrementalScan that = (SparkIncrementalScan) other; + return table().name().equals(that.table().name()) + && Objects.equals(table().uuid(), that.table().uuid()) + && startSnapshotId == that.startSnapshotId + && Objects.equals(endSnapshotId, that.endSnapshotId) + && readSchema().equals(that.readSchema()) // compare Spark schemas to ignore field ids + && filtersDesc().equals(that.filtersDesc()) + && runtimeFiltersDesc().equals(that.runtimeFiltersDesc()); + } + + @Override + public int hashCode() { + return Objects.hash( + table().name(), + table().uuid(), + startSnapshotId, + endSnapshotId, + readSchema(), + filtersDesc(), + runtimeFiltersDesc()); + } + + @Override + public String description() { + return String.format( + "IcebergIncrementalScan(table=%s, startSnapshotId=%s, endSnapshotId=%s, filters=%s, runtimeFilters=%s, groupedBy=%s)", + table(), + startSnapshotId, + endSnapshotId, + filtersDesc(), + runtimeFiltersDesc(), + groupingKeyDesc()); + } +} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java index 8e3da5f57b5f..98a0061b3a33 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java @@ -34,25 +34,25 @@ class SparkInputPartition implements InputPartition, HasPartitionKey, Serializab private final Types.StructType groupingKeyType; private final ScanTaskGroup taskGroup; private final Broadcast
tableBroadcast; - private final String expectedSchemaString; + private final String projectionString; private final boolean caseSensitive; private final transient String[] preferredLocations; private final boolean cacheDeleteFilesOnExecutors; - private transient Schema expectedSchema = null; + private transient Schema projection = null; SparkInputPartition( Types.StructType groupingKeyType, ScanTaskGroup taskGroup, Broadcast
tableBroadcast, - String expectedSchemaString, + String projectionString, boolean caseSensitive, String[] preferredLocations, boolean cacheDeleteFilesOnExecutors) { this.groupingKeyType = groupingKeyType; this.taskGroup = taskGroup; this.tableBroadcast = tableBroadcast; - this.expectedSchemaString = expectedSchemaString; + this.projectionString = projectionString; this.caseSensitive = caseSensitive; this.preferredLocations = preferredLocations; this.cacheDeleteFilesOnExecutors = cacheDeleteFilesOnExecutors; @@ -89,11 +89,11 @@ public boolean cacheDeleteFilesOnExecutors() { return cacheDeleteFilesOnExecutors; } - public Schema expectedSchema() { - if (expectedSchema == null) { - this.expectedSchema = SchemaParser.fromJson(expectedSchemaString); + public Schema projection() { + if (projection == null) { + this.projection = SchemaParser.fromJson(projectionString); } - return expectedSchema; + return projection; } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java index a70176253bb9..dff844eb45c9 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java @@ -45,7 +45,6 @@ import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; -import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.TableScanUtil; import org.apache.spark.sql.SparkSession; @@ -74,12 +73,13 @@ abstract class SparkPartitioningAwareScan extends S SparkPartitioningAwareScan( SparkSession spark, Table table, + Schema schema, Scan> scan, SparkReadConf readConf, - Schema expectedSchema, + Schema projection, List filters, Supplier scanReportSupplier) { - super(spark, table, readConf, expectedSchema, filters, scanReportSupplier); + super(spark, table, schema, readConf, projection, filters, scanReportSupplier); this.scan = scan; this.preserveDataGrouping = readConf.preserveDataGrouping(); @@ -129,7 +129,7 @@ protected StructType groupingKeyType() { } private StructType computeGroupingKeyType() { - return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs()); + return org.apache.iceberg.Partitioning.groupingKeyType(projection(), specs()); } private Transform[] groupingKeyTransforms() { @@ -141,8 +141,7 @@ private Transform[] groupingKeyTransforms() { .map(field -> fieldsById.get(field.fieldId())) .collect(Collectors.toList()); - Schema schema = SnapshotUtil.schemaFor(table(), branch()); - this.groupingKeyTransforms = Spark3Util.toTransforms(schema, groupingKeyFields); + this.groupingKeyTransforms = Spark3Util.toTransforms(schema(), groupingKeyFields); } return groupingKeyTransforms; diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java index 97f9601cedd9..5826ee1d9508 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java @@ -21,6 +21,7 @@ import java.util.List; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableUtil; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -40,6 +41,7 @@ class SparkPositionDeltaOperation implements RowLevelOperation, SupportsDelta { private final SparkSession spark; private final Table table; + private final Snapshot snapshot; private final String branch; private final Command command; private final IsolationLevel isolationLevel; @@ -52,11 +54,13 @@ class SparkPositionDeltaOperation implements RowLevelOperation, SupportsDelta { SparkPositionDeltaOperation( SparkSession spark, Table table, + Snapshot snapshot, String branch, RowLevelOperationInfo info, IsolationLevel isolationLevel) { this.spark = spark; this.table = table; + this.snapshot = snapshot; this.branch = branch; this.command = info.command(); this.isolationLevel = isolationLevel; @@ -71,10 +75,10 @@ public Command command() { public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { if (lazyScanBuilder == null) { this.lazyScanBuilder = - new SparkScanBuilder(spark, table, branch, options) { + new SparkScanBuilder(spark, table, table.schema(), snapshot, branch, options) { @Override public Scan build() { - Scan scan = super.buildMergeOnReadScan(); + Scan scan = super.build(); SparkPositionDeltaOperation.this.configuredScan = scan; return scan; } @@ -99,16 +103,16 @@ public DeltaWriteBuilder newWriteBuilder(LogicalWriteInfo info) { @Override public NamedReference[] requiredMetadataAttributes() { - List metadataAttributes = Lists.newArrayList(); - metadataAttributes.add(Expressions.column(MetadataColumns.SPEC_ID.name())); - metadataAttributes.add(Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME)); + List metaAttrs = Lists.newArrayList(); + metaAttrs.add(Expressions.column(MetadataColumns.SPEC_ID.name())); + metaAttrs.add(Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME)); + if (TableUtil.supportsRowLineage(table)) { - metadataAttributes.add(Expressions.column(MetadataColumns.ROW_ID.name())); - metadataAttributes.add( - Expressions.column(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name())); + metaAttrs.add(Expressions.column(MetadataColumns.ROW_ID.name())); + metaAttrs.add(Expressions.column(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name())); } - return metadataAttributes.toArray(new NamedReference[0]); + return metaAttrs.toArray(new NamedReference[0]); } @Override diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index 6f091eb2a471..763d54dd1bb7 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -52,7 +52,6 @@ import org.apache.iceberg.encryption.EncryptingFileIO; import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.BasePositionDeltaWriter; import org.apache.iceberg.io.ClusteredDataWriter; import org.apache.iceberg.io.ClusteredPositionDeleteWriter; @@ -106,13 +105,13 @@ class SparkPositionDeltaWrite extends BaseSparkWrite private final JavaSparkContext sparkContext; private final Table table; + private final String branch; private final Command command; private final SparkBatchQueryScan scan; private final IsolationLevel isolationLevel; private final String applicationId; private final boolean wapEnabled; private final String wapId; - private final String branch; private final Map extraSnapshotMetadata; private final SparkWriteRequirements writeRequirements; private final Context context; @@ -123,6 +122,7 @@ class SparkPositionDeltaWrite extends BaseSparkWrite SparkPositionDeltaWrite( SparkSession spark, Table table, + String branch, Command command, SparkBatchQueryScan scan, IsolationLevel isolationLevel, @@ -131,13 +131,13 @@ class SparkPositionDeltaWrite extends BaseSparkWrite Schema dataSchema) { this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); this.table = table; + this.branch = branch; this.command = command; this.scan = scan; this.isolationLevel = isolationLevel; this.applicationId = spark.sparkContext().applicationId(); this.wapEnabled = writeConf.wapEnabled(); this.wapId = writeConf.wapId(); - this.branch = writeConf.branch(); this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata(); this.writeRequirements = writeConf.positionDeltaRequirements(command); this.context = new Context(dataSchema, writeConf, info, writeRequirements); @@ -247,7 +247,7 @@ public void commit(WriterCommitMessage[] messages, WriteSummary summary) { // the scan may be null if the optimizer replaces it with an empty relation // no validation is needed in this case as the command is independent of the table state if (scan != null) { - Expression conflictDetectionFilter = conflictDetectionFilter(scan); + Expression conflictDetectionFilter = scan.filter(); rowDelta.conflictDetectionFilter(conflictDetectionFilter); rowDelta.validateDataFilesExist(referencedDataFiles); @@ -291,16 +291,6 @@ public void commit(WriterCommitMessage[] messages, WriteSummary summary) { } } - private Expression conflictDetectionFilter(SparkBatchQueryScan queryScan) { - Expression filter = Expressions.alwaysTrue(); - - for (Expression expr : queryScan.filterExpressions()) { - filter = Expressions.and(filter, expr); - } - - return filter; - } - @Override public void abort(WriterCommitMessage[] messages) { if (cleanupOnAbort) { diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java index 9151e41c0cf9..87379126a646 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java @@ -29,6 +29,7 @@ import org.apache.iceberg.TableUtil; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.spark.SparkWriteConf; import org.apache.iceberg.types.TypeUtil; @@ -48,6 +49,7 @@ class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder { private final SparkSession spark; private final Table table; + private final String branch; private final Command command; private final SparkBatchQueryScan scan; private final IsolationLevel isolationLevel; @@ -66,13 +68,15 @@ class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder { LogicalWriteInfo info) { this.spark = spark; this.table = table; + this.branch = branch; this.command = command; this.scan = (SparkBatchQueryScan) scan; this.isolationLevel = isolationLevel; - this.writeConf = new SparkWriteConf(spark, table, branch, info.options()); + this.writeConf = new SparkWriteConf(spark, table, info.options()); this.info = info; this.checkNullability = writeConf.checkNullability(); this.checkOrdering = writeConf.checkOrdering(); + SparkTableUtil.validateWriteBranch(spark, table, branch, info.options()); } @Override @@ -93,7 +97,7 @@ public DeltaWrite build() { SparkUtil.validatePartitionTransforms(table.spec()); return new SparkPositionDeltaWrite( - spark, table, command, scan, isolationLevel, writeConf, info, dataSchema); + spark, table, branch, command, scan, isolationLevel, writeConf, info, dataSchema); } private Schema dataSchema() { diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRewriteWriteBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRewriteWriteBuilder.java index 714ac8e4853e..c605b2ce91d8 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRewriteWriteBuilder.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRewriteWriteBuilder.java @@ -64,6 +64,7 @@ public Write build() { return new SparkWrite( spark, table, + null /* main branch */, writeConf, info, appId, diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java index b113bd9b25af..8d763c7c6a84 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java @@ -34,6 +34,7 @@ import java.util.Map; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.write.RowLevelOperation; @@ -45,15 +46,21 @@ class SparkRowLevelOperationBuilder implements RowLevelOperationBuilder { private final SparkSession spark; private final Table table; + private final Snapshot snapshot; private final String branch; private final RowLevelOperationInfo info; private final RowLevelOperationMode mode; private final IsolationLevel isolationLevel; SparkRowLevelOperationBuilder( - SparkSession spark, Table table, String branch, RowLevelOperationInfo info) { + SparkSession spark, + Table table, + Snapshot snapshot, + String branch, + RowLevelOperationInfo info) { this.spark = spark; this.table = table; + this.snapshot = snapshot; this.branch = branch; this.info = info; this.mode = mode(table.properties(), info.command()); @@ -62,14 +69,12 @@ class SparkRowLevelOperationBuilder implements RowLevelOperationBuilder { @Override public RowLevelOperation build() { - switch (mode) { - case COPY_ON_WRITE: - return new SparkCopyOnWriteOperation(spark, table, branch, info, isolationLevel); - case MERGE_ON_READ: - return new SparkPositionDeltaOperation(spark, table, branch, info, isolationLevel); - default: - throw new IllegalArgumentException("Unsupported operation mode: " + mode); - } + return switch (mode) { + case COPY_ON_WRITE -> + new SparkCopyOnWriteOperation(spark, table, snapshot, branch, info, isolationLevel); + case MERGE_ON_READ -> + new SparkPositionDeltaOperation(spark, table, snapshot, branch, info, isolationLevel); + }; } private RowLevelOperationMode mode(Map properties, Command command) { diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRuntimeFilterableScan.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRuntimeFilterableScan.java new file mode 100644 index 000000000000..c626e04eb038 --- /dev/null +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRuntimeFilterableScan.java @@ -0,0 +1,200 @@ +/* + * 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 java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Scan; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionUtil; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Projections; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkV2Filters; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.iceberg.util.DeleteFileSet; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.expressions.filter.Predicate; +import org.apache.spark.sql.connector.read.SupportsRuntimeV2Filtering; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class SparkRuntimeFilterableScan extends SparkPartitioningAwareScan + implements SupportsRuntimeV2Filtering { + + private static final Logger LOG = LoggerFactory.getLogger(SparkRuntimeFilterableScan.class); + + private final List runtimeFilterExpressions; + + protected SparkRuntimeFilterableScan( + SparkSession spark, + Table table, + Schema schema, + Scan> scan, + SparkReadConf readConf, + Schema projection, + List filters, + Supplier scanReportSupplier) { + super(spark, table, schema, scan, readConf, projection, filters, scanReportSupplier); + this.runtimeFilterExpressions = Lists.newArrayList(); + } + + @Override + protected Class taskJavaClass() { + return PartitionScanTask.class; + } + + @Override + public NamedReference[] filterAttributes() { + Set partitionFieldSourceIds = Sets.newHashSet(); + + for (PartitionSpec spec : specs()) { + for (PartitionField field : spec.fields()) { + partitionFieldSourceIds.add(field.sourceId()); + } + } + + Map quotedNameById = SparkSchemaUtil.indexQuotedNameById(projection()); + + // the optimizer will look for an equality condition with filter attributes in a join + // as the scan has been already planned, filtering can only be done on projected attributes + // that's why only partition source fields that are part of the read schema can be reported + + return partitionFieldSourceIds.stream() + .filter(fieldId -> projection().findField(fieldId) != null) + .map(fieldId -> Spark3Util.toNamedReference(quotedNameById.get(fieldId))) + .toArray(NamedReference[]::new); + } + + @Override + public void filter(Predicate[] predicates) { + Expression runtimeFilterExpr = convertRuntimeFilters(predicates); + + if (runtimeFilterExpr != Expressions.alwaysTrue()) { + Map evaluatorsBySpecId = Maps.newHashMap(); + + for (PartitionSpec spec : specs()) { + Expression inclusiveExpr = + Projections.inclusive(spec, caseSensitive()).project(runtimeFilterExpr); + Evaluator inclusive = new Evaluator(spec.partitionType(), inclusiveExpr); + evaluatorsBySpecId.put(spec.specId(), inclusive); + } + + List filteredTasks = + tasks().stream() + .filter( + task -> { + Evaluator evaluator = evaluatorsBySpecId.get(task.spec().specId()); + return evaluator.eval(task.partition()); + }) + .collect(Collectors.toList()); + + LOG.info( + "{} of {} task(s) for table {} matched runtime filter {}", + filteredTasks.size(), + tasks().size(), + table().name(), + ExpressionUtil.toSanitizedString(runtimeFilterExpr)); + + // don't invalidate tasks if the runtime filter had no effect to avoid planning splits again + if (filteredTasks.size() < tasks().size()) { + resetTasks(filteredTasks); + } + + // save the evaluated filter for equals/hashCode + runtimeFilterExpressions.add(runtimeFilterExpr); + } + } + + protected Map rewritableDeletes(boolean forDVs) { + Map rewritableDeletes = Maps.newHashMap(); + + for (ScanTask task : tasks()) { + FileScanTask fileScanTask = task.asFileScanTask(); + for (DeleteFile deleteFile : fileScanTask.deletes()) { + if (shouldRewrite(deleteFile, forDVs)) { + rewritableDeletes + .computeIfAbsent(fileScanTask.file().location(), ignored -> DeleteFileSet.create()) + .add(deleteFile); + } + } + } + + return rewritableDeletes; + } + + // for DVs all position deletes must be rewritten + // for position deletes, only file-scoped deletes must be rewritten + private boolean shouldRewrite(DeleteFile deleteFile, boolean forDVs) { + if (forDVs) { + return deleteFile.content() != FileContent.EQUALITY_DELETES; + } + + return ContentFileUtil.isFileScoped(deleteFile); + } + + // at this moment, Spark can only pass IN filters for a single attribute + // if there are multiple filter attributes, Spark will pass two separate IN filters + private Expression convertRuntimeFilters(Predicate[] predicates) { + Expression runtimeFilterExpr = Expressions.alwaysTrue(); + + for (Predicate predicate : predicates) { + Expression expr = SparkV2Filters.convert(predicate); + if (expr != null) { + try { + Binder.bind(projection().asStruct(), expr, caseSensitive()); + runtimeFilterExpr = Expressions.and(runtimeFilterExpr, expr); + } catch (ValidationException e) { + LOG.warn("Failed to bind {} to expected schema, skipping runtime filter", expr, e); + } + } else { + LOG.warn("Unsupported runtime filter {}", predicate); + } + } + + return runtimeFilterExpr; + } + + protected String runtimeFiltersDesc() { + return Spark3Util.describe(runtimeFilterExpressions); + } +} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 14fe80c43daf..879c2a01f7de 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -33,8 +33,10 @@ import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; @@ -76,7 +78,6 @@ import org.apache.iceberg.spark.source.metrics.TotalPlanningDuration; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.TableScanUtil; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; @@ -101,12 +102,12 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { private final JavaSparkContext sparkContext; private final Table table; + private final Schema schema; private final SparkSession spark; private final SparkReadConf readConf; private final boolean caseSensitive; - private final Schema expectedSchema; - private final List filterExpressions; - private final String branch; + private final Schema projection; + private final List filters; private final Supplier scanReportSupplier; // lazy variables @@ -115,46 +116,45 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { SparkScan( SparkSession spark, Table table, + Schema schema, SparkReadConf readConf, - Schema expectedSchema, + Schema projection, List filters, Supplier scanReportSupplier) { - Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch()); - SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema); - this.spark = spark; this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); this.table = table; + this.schema = schema; this.readConf = readConf; this.caseSensitive = readConf.caseSensitive(); - this.expectedSchema = expectedSchema; - this.filterExpressions = filters != null ? filters : Collections.emptyList(); - this.branch = readConf.branch(); + this.projection = projection; + this.filters = filters != null ? filters : ImmutableList.of(); this.scanReportSupplier = scanReportSupplier; + SparkSchemaUtil.validateMetadataColumnReferences(schema, projection); } protected Table table() { return table; } - protected String branch() { - return branch; - } - protected boolean caseSensitive() { return caseSensitive; } - protected Schema expectedSchema() { - return expectedSchema; + protected Schema schema() { + return schema; } - protected List filterExpressions() { - return filterExpressions; + protected Schema projection() { + return projection; + } + + protected Expression filter() { + return filters.stream().reduce(Expressions.alwaysTrue(), Expressions::and); } protected String filtersDesc() { - return Spark3Util.describe(filterExpressions); + return Spark3Util.describe(filters); } protected Types.StructType groupingKeyType() { @@ -166,26 +166,36 @@ protected Types.StructType groupingKeyType() { @Override public Batch toBatch() { return new SparkBatch( - sparkContext, table, readConf, groupingKeyType(), taskGroups(), expectedSchema, hashCode()); + sparkContext, table, readConf, groupingKeyType(), taskGroups(), projection, hashCode()); } @Override public MicroBatchStream toMicroBatchStream(String checkpointLocation) { - return new SparkMicroBatchStream( - sparkContext, table, readConf, expectedSchema, checkpointLocation); + return new SparkMicroBatchStream(sparkContext, table, readConf, projection, checkpointLocation); } @Override public StructType readSchema() { if (readSchema == null) { - this.readSchema = SparkSchemaUtil.convert(expectedSchema); + this.readSchema = SparkSchemaUtil.convert(projection); } return readSchema; } + /** + * Reports the default stats. + * + *

Note that the default implementation is based on task groups and read schema and is good + * enough when the scan doesn't apply to a particular snapshot (e.g. incremental scan). Regular + * batch scans are expected to override this behavior and also leverage snapshot information. + * + * @return the default stats estimates + */ @Override public Statistics estimateStatistics() { - return estimateStatistics(SnapshotUtil.latestSnapshot(table, branch)); + long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum(); + long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount); + return new Stats(sizeInBytes, rowsCount, Collections.emptyMap()); } protected Statistics estimateStatistics(Snapshot snapshot) { @@ -240,7 +250,7 @@ protected Statistics estimateStatistics(Snapshot snapshot) { // estimate stats using snapshot summary only for partitioned tables // (metadata tables are unpartitioned) - if (!table.spec().isUnpartitioned() && filterExpressions.isEmpty()) { + if (!table.spec().isUnpartitioned() && filters.isEmpty()) { LOG.debug( "Using snapshot {} metadata to estimate statistics for table {}", snapshot.snapshotId(), diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 2fb188d83d43..2ac070a7cb11 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -20,178 +20,107 @@ import java.io.IOException; import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; +import java.util.Objects; +import org.apache.iceberg.BaseMetadataTable; import org.apache.iceberg.BaseTable; import org.apache.iceberg.BatchScan; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.IncrementalAppendScan; -import org.apache.iceberg.IncrementalChangelogScan; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.MetricsModes; +import org.apache.iceberg.ScanTask; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SparkDistributedDataScan; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.expressions.AggregateEvaluator; import org.apache.iceberg.expressions.Binder; import org.apache.iceberg.expressions.BoundAggregate; import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.ExpressionUtil; -import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.metrics.InMemoryMetricsReporter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkAggregates; -import org.apache.iceberg.spark.SparkReadConf; -import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.SparkV2Filters; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.TimeTravel; import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.TypeUtil.GetID; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.Pair; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.expressions.aggregate.AggregateFunc; import org.apache.spark.sql.connector.expressions.aggregate.Aggregation; -import org.apache.spark.sql.connector.expressions.filter.Predicate; import org.apache.spark.sql.connector.read.Scan; -import org.apache.spark.sql.connector.read.ScanBuilder; -import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsPushDownAggregates; import org.apache.spark.sql.connector.read.SupportsPushDownLimit; import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns; import org.apache.spark.sql.connector.read.SupportsPushDownV2Filters; -import org.apache.spark.sql.connector.read.SupportsReportStatistics; -import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class SparkScanBuilder - implements ScanBuilder, - SupportsPushDownAggregates, - SupportsPushDownV2Filters, +public class SparkScanBuilder extends BaseSparkScanBuilder + implements SupportsPushDownV2Filters, SupportsPushDownRequiredColumns, - SupportsReportStatistics, - SupportsPushDownLimit { + SupportsPushDownLimit, + SupportsPushDownAggregates { private static final Logger LOG = LoggerFactory.getLogger(SparkScanBuilder.class); - private static final Predicate[] NO_PREDICATES = new Predicate[0]; - private Scan localScan; - private final SparkSession spark; - private final Table table; - private final CaseInsensitiveStringMap options; - private final SparkReadConf readConf; - private final Set metaFieldNames = Sets.newLinkedHashSet(); - private final InMemoryMetricsReporter metricsReporter; + private final Snapshot snapshot; + private final String branch; + private final TimeTravel timeTravel; + private final Long startSnapshotId; + private final Long endSnapshotId; + private Scan localScan; - private Schema projection; - private boolean caseSensitive; - private List filterExpressions = null; - private Predicate[] pushedPredicates = NO_PREDICATES; - private Integer limit = null; + SparkScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { + this( + spark, + table, + table.schema(), + table.currentSnapshot(), + null /* no branch */, + null /* no time travel */, + options); + } SparkScanBuilder( SparkSession spark, Table table, - String branch, Schema schema, + Snapshot snapshot, + String branch, CaseInsensitiveStringMap options) { - this.spark = spark; - this.table = table; - this.projection = schema; - this.options = options; - this.readConf = new SparkReadConf(spark, table, branch, options); - this.caseSensitive = readConf.caseSensitive(); - this.metricsReporter = new InMemoryMetricsReporter(); - } - - SparkScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { - this(spark, table, table.schema(), options); + this(spark, table, schema, snapshot, branch, null /* no time travel */, options); } SparkScanBuilder( - SparkSession spark, Table table, String branch, CaseInsensitiveStringMap options) { - this(spark, table, branch, SnapshotUtil.schemaFor(table, branch), options); - } - - SparkScanBuilder( - SparkSession spark, Table table, Schema schema, CaseInsensitiveStringMap options) { - this(spark, table, null, schema, options); - } - - private Expression filterExpression() { - if (filterExpressions != null) { - return filterExpressions.stream().reduce(Expressions.alwaysTrue(), Expressions::and); - } - return Expressions.alwaysTrue(); - } - - public SparkScanBuilder caseSensitive(boolean isCaseSensitive) { - this.caseSensitive = isCaseSensitive; - return this; - } - - @Override - public Predicate[] pushPredicates(Predicate[] predicates) { - // there are 3 kinds of filters: - // (1) filters that can be pushed down completely and don't have to evaluated by Spark - // (e.g. filters that select entire partitions) - // (2) filters that can be pushed down partially and require record-level filtering in Spark - // (e.g. filters that may select some but not necessarily all rows in a file) - // (3) filters that can't be pushed down at all and have to be evaluated by Spark - // (e.g. unsupported filters) - // filters (1) and (2) are used prune files during job planning in Iceberg - // filters (2) and (3) form a set of post scan filters and must be evaluated by Spark - - List expressions = Lists.newArrayListWithExpectedSize(predicates.length); - List pushableFilters = Lists.newArrayListWithExpectedSize(predicates.length); - List postScanFilters = Lists.newArrayListWithExpectedSize(predicates.length); - - for (Predicate predicate : predicates) { - try { - Expression expr = SparkV2Filters.convert(predicate); - - if (expr != null) { - // try binding the expression to ensure it can be pushed down - Binder.bind(projection.asStruct(), expr, caseSensitive); - expressions.add(expr); - pushableFilters.add(predicate); - } - - if (expr == null || !ExpressionUtil.selectsPartitions(expr, table, caseSensitive)) { - postScanFilters.add(predicate); - } else { - LOG.info("Evaluating completely on Iceberg side: {}", predicate); - } - - } catch (Exception e) { - LOG.warn("Failed to check if {} can be pushed down: {}", predicate, e.getMessage()); - postScanFilters.add(predicate); - } + SparkSession spark, + Table table, + Schema schema, + Snapshot snapshot, + String branch, + TimeTravel timeTravel, + CaseInsensitiveStringMap options) { + super(spark, table, schema, options); + this.snapshot = snapshot; + this.branch = branch; + this.timeTravel = timeTravel; + if (Spark3Util.containsIncrementalOptions(options)) { + Preconditions.checkArgument(timeTravel == null, "Cannot use time travel in incremental scan"); + Pair boundaries = readConf().incrementalAppendScanBoundaries(); + this.startSnapshotId = boundaries.first(); + this.endSnapshotId = boundaries.second(); + } else { + this.startSnapshotId = null; + this.endSnapshotId = null; } - - this.filterExpressions = expressions; - this.pushedPredicates = pushableFilters.toArray(new Predicate[0]); - - return postScanFilters.toArray(new Predicate[0]); - } - - @Override - public Predicate[] pushedPredicates() { - return pushedPredicates; + Spark3Util.validateNoLegacyTimeTravel(options); + SparkTableUtil.validateReadBranch(spark, table, branch, options); } @Override @@ -208,7 +137,7 @@ public boolean pushAggregation(Aggregation aggregation) { try { Expression expr = SparkAggregates.convert(aggregateFunc); if (expr != null) { - Expression bound = Binder.bind(projection.asStruct(), expr, caseSensitive); + Expression bound = Binder.bind(projection().asStruct(), expr, caseSensitive()); expressions.add((BoundAggregate) bound); } else { LOG.info( @@ -228,10 +157,7 @@ public boolean pushAggregation(Aggregation aggregation) { return false; } - org.apache.iceberg.Scan scan = - buildIcebergBatchScan(true /* include Column Stats */, projectionWithMetadataColumns()); - - try (CloseableIterable fileScanTasks = scan.planFiles()) { + try (CloseableIterable fileScanTasks = planFilesWithStats()) { for (FileScanTask task : fileScanTasks) { if (!task.deletes().isEmpty()) { LOG.info("Skipping aggregate pushdown: detected row level deletes"); @@ -255,18 +181,17 @@ public boolean pushAggregation(Aggregation aggregation) { StructLike structLike = aggregateEvaluator.result(); pushedAggregateRows[0] = new StructInternalRow(aggregateEvaluator.resultType()).setStruct(structLike); - localScan = - new SparkLocalScan(table, pushedAggregateSchema, pushedAggregateRows, filterExpressions); + localScan = new SparkLocalScan(table(), pushedAggregateSchema, pushedAggregateRows, filters()); return true; } private boolean canPushDownAggregation(Aggregation aggregation) { - if (!(table instanceof BaseTable)) { + if (!isMainTable()) { return false; } - if (!readConf.aggregatePushDownEnabled()) { + if (!readConf().aggregatePushDownEnabled()) { return false; } @@ -282,7 +207,7 @@ private boolean canPushDownAggregation(Aggregation aggregation) { } private boolean metricsModeSupportsAggregatePushDown(List> aggregates) { - MetricsConfig config = MetricsConfig.forTable(table); + MetricsConfig config = MetricsConfig.forTable(table()); for (BoundAggregate aggregate : aggregates) { String colName = aggregate.columnName(); if (!colName.equals("*")) { @@ -316,182 +241,89 @@ private boolean metricsModeSupportsAggregatePushDown(List> return true; } - @Override - public void pruneColumns(StructType requestedSchema) { - List dataFields = Lists.newArrayList(); - - for (StructField field : requestedSchema.fields()) { - if (MetadataColumns.isMetadataColumn(field.name())) { - metaFieldNames.add(field.name()); - } else { - dataFields.add(field); - } - } - - StructType requestedProjection = SparkSchemaUtil.toStructType(dataFields); - this.projection = prune(projection, requestedProjection); - } - - // the projection should include all columns that will be returned, - // including those only used in filters - private Schema prune(Schema schema, StructType requestedSchema) { - return SparkSchemaUtil.prune(schema, requestedSchema, filterExpression(), caseSensitive); - } - - // schema of rows that must be returned by readers - protected Schema projectionWithMetadataColumns() { - return TypeUtil.join(projection, calculateMetadataSchema()); - } - - // computes metadata schema avoiding conflicts between partition and data field IDs - private Schema calculateMetadataSchema() { - List metaFields = metaFields(); - Optional partitionField = findPartitionField(metaFields); - - if (partitionField.isEmpty()) { - return new Schema(metaFields); - } - - Types.StructType partitionType = partitionField.get().type().asStructType(); - Set partitionFieldIds = TypeUtil.getProjectedIds(partitionType); - GetID getId = TypeUtil.reassignConflictingIds(partitionFieldIds, allUsedFieldIds()); - return new Schema(metaFields, getId); - } - - private List metaFields() { - return metaFieldNames.stream() - .map(name -> MetadataColumns.metadataColumn(table, name)) - .collect(Collectors.toList()); - } - - private Optional findPartitionField(List fields) { - return fields.stream() - .filter(field -> MetadataColumns.PARTITION_COLUMN_ID == field.fieldId()) - .findFirst(); - } - - // collects used data field IDs across all known table schemas - private Set allUsedFieldIds() { - return table.schemas().values().stream() - .flatMap(tableSchema -> TypeUtil.getProjectedIds(tableSchema.asStruct()).stream()) - .collect(Collectors.toSet()); - } - @Override public Scan build() { if (localScan != null) { return localScan; + } else if (startSnapshotId != null) { + return buildIncrementalAppendScan(); } else { return buildBatchScan(); } } private Scan buildBatchScan() { - Schema expectedSchema = projectionWithMetadataColumns(); + Schema projection = projectionWithMetadataColumns(); return new SparkBatchQueryScan( - spark, - table, - buildIcebergBatchScan(false /* not include Column Stats */, expectedSchema), - readConf, - expectedSchema, - filterExpressions, - metricsReporter::scanReport); + spark(), + table(), + schema(), + snapshot, + branch, + buildIcebergBatchScan(projection, false /* use residuals */, false /* no stats */), + readConf(), + projection, + filters(), + metricsReporter()::scanReport); + } + + private Scan buildIncrementalAppendScan() { + Schema projection = projectionWithMetadataColumns(); + return new SparkIncrementalScan( + spark(), + table(), + startSnapshotId, + endSnapshotId, + buildIcebergIncrementalAppendScan(projection, false /* no stats */), + readConf(), + projection, + filters(), + metricsReporter()::scanReport); } - private org.apache.iceberg.Scan buildIcebergBatchScan(boolean withStats, Schema expectedSchema) { - Long snapshotId = readConf.snapshotId(); - Long asOfTimestamp = readConf.asOfTimestamp(); - String branch = readConf.branch(); - String tag = readConf.tag(); - - Preconditions.checkArgument( - snapshotId == null || asOfTimestamp == null, - "Cannot set both %s and %s to select which table snapshot to scan", - SparkReadOptions.SNAPSHOT_ID, - SparkReadOptions.AS_OF_TIMESTAMP); - - Long startSnapshotId = readConf.startSnapshotId(); - Long endSnapshotId = readConf.endSnapshotId(); - - if (snapshotId != null || asOfTimestamp != null) { - Preconditions.checkArgument( - startSnapshotId == null && endSnapshotId == null, - "Cannot set %s and %s for incremental scans when either %s or %s is set", - SparkReadOptions.START_SNAPSHOT_ID, - SparkReadOptions.END_SNAPSHOT_ID, - SparkReadOptions.SNAPSHOT_ID, - SparkReadOptions.AS_OF_TIMESTAMP); - } - - Preconditions.checkArgument( - startSnapshotId != null || endSnapshotId == null, - "Cannot set only %s for incremental scans. Please, set %s too.", - SparkReadOptions.END_SNAPSHOT_ID, - SparkReadOptions.START_SNAPSHOT_ID); - - Long startTimestamp = readConf.startTimestamp(); - Long endTimestamp = readConf.endTimestamp(); - Preconditions.checkArgument( - startTimestamp == null && endTimestamp == null, - "Cannot set %s or %s for incremental scans and batch scan. They are only valid for " - + "changelog scans.", - SparkReadOptions.START_TIMESTAMP, - SparkReadOptions.END_TIMESTAMP); - - if (startSnapshotId != null) { - return buildIncrementalAppendScan(startSnapshotId, endSnapshotId, withStats, expectedSchema); + public Scan buildCopyOnWriteScan() { + Schema projection = projectionWithMetadataColumns(); + return new SparkCopyOnWriteScan( + spark(), + table(), + schema(), + snapshot, + branch, + buildIcebergBatchScan(projection, true /* ignore residuals */, false /* no stats */), + readConf(), + projection, + filters(), + metricsReporter()::scanReport); + } + + private CloseableIterable planFilesWithStats() { + Schema projection = projectionWithMetadataColumns(); + org.apache.iceberg.Scan scan = buildIcebergScanWithStats(projection); + if (scan != null) { + return CloseableIterable.transform(scan.planFiles(), ScanTask::asFileScanTask); } else { - return buildBatchScan(snapshotId, asOfTimestamp, branch, tag, withStats, expectedSchema); + return CloseableIterable.empty(); } } - private org.apache.iceberg.Scan buildBatchScan( - Long snapshotId, - Long asOfTimestamp, - String branch, - String tag, - boolean withStats, - Schema expectedSchema) { - BatchScan scan = - newBatchScan() - .caseSensitive(caseSensitive) - .filter(filterExpression()) - .project(expectedSchema) - .metricsReporter(metricsReporter); - - if (withStats) { - scan = scan.includeColumnStats(); - } - - if (snapshotId != null) { - scan = scan.useSnapshot(snapshotId); - } - - if (asOfTimestamp != null) { - scan = scan.asOfTime(asOfTimestamp); - } - - if (branch != null) { - scan = scan.useRef(branch); - } - - if (tag != null) { - scan = scan.useRef(tag); + private org.apache.iceberg.Scan buildIcebergScanWithStats(Schema projection) { + if (startSnapshotId != null) { + return buildIcebergIncrementalAppendScan(projection, true /* with stats */); + } else { + return buildIcebergBatchScan(projection, false /* use residuals */, true /* with stats */); } - - return configureSplitPlanning(scan); } - private org.apache.iceberg.Scan buildIncrementalAppendScan( - long startSnapshotId, Long endSnapshotId, boolean withStats, Schema expectedSchema) { + private IncrementalAppendScan buildIcebergIncrementalAppendScan( + Schema projection, boolean withStats) { IncrementalAppendScan scan = - table + table() .newIncrementalAppendScan() .fromSnapshotExclusive(startSnapshotId) - .caseSensitive(caseSensitive) - .filter(filterExpression()) - .project(expectedSchema) - .metricsReporter(metricsReporter); + .caseSensitive(caseSensitive()) + .filter(filter()) + .project(projection) + .metricsReporter(metricsReporter()); if (withStats) { scan = scan.includeColumnStats(); @@ -504,251 +336,61 @@ private org.apache.iceberg.Scan buildIncrementalAppendScan( return configureSplitPlanning(scan); } - @SuppressWarnings("CyclomaticComplexity") - public Scan buildChangelogScan() { - Preconditions.checkArgument( - readConf.snapshotId() == null - && readConf.asOfTimestamp() == null - && readConf.branch() == null - && readConf.tag() == null, - "Cannot set neither %s, %s, %s and %s for changelogs", - SparkReadOptions.SNAPSHOT_ID, - SparkReadOptions.AS_OF_TIMESTAMP, - SparkReadOptions.BRANCH, - SparkReadOptions.TAG); - - Long startSnapshotId = readConf.startSnapshotId(); - Long endSnapshotId = readConf.endSnapshotId(); - Long startTimestamp = readConf.startTimestamp(); - Long endTimestamp = readConf.endTimestamp(); - - Preconditions.checkArgument( - !(startSnapshotId != null && startTimestamp != null), - "Cannot set both %s and %s for changelogs", - SparkReadOptions.START_SNAPSHOT_ID, - SparkReadOptions.START_TIMESTAMP); - - Preconditions.checkArgument( - !(endSnapshotId != null && endTimestamp != null), - "Cannot set both %s and %s for changelogs", - SparkReadOptions.END_SNAPSHOT_ID, - SparkReadOptions.END_TIMESTAMP); - - if (startTimestamp != null && endTimestamp != null) { - Preconditions.checkArgument( - startTimestamp < endTimestamp, - "Cannot set %s to be greater than %s for changelogs", - SparkReadOptions.START_TIMESTAMP, - SparkReadOptions.END_TIMESTAMP); - } - - boolean emptyScan = false; - if (startTimestamp != null) { - if (table.currentSnapshot() == null - || startTimestamp > table.currentSnapshot().timestampMillis()) { - emptyScan = true; - } - startSnapshotId = getStartSnapshotId(startTimestamp); - } - - if (endTimestamp != null) { - endSnapshotId = getEndSnapshotId(endTimestamp); - if ((startSnapshotId == null && endSnapshotId == null) - || (startSnapshotId != null && startSnapshotId.equals(endSnapshotId))) { - emptyScan = true; - } - } - - Schema expectedSchema = projectionWithMetadataColumns(); - - IncrementalChangelogScan scan = - table - .newIncrementalChangelogScan() - .caseSensitive(caseSensitive) - .filter(filterExpression()) - .project(expectedSchema) - .metricsReporter(metricsReporter); - - if (startSnapshotId != null) { - scan = scan.fromSnapshotExclusive(startSnapshotId); - } - - if (endSnapshotId != null) { - scan = scan.toSnapshot(endSnapshotId); - } - - scan = configureSplitPlanning(scan); - - return new SparkChangelogScan( - spark, table, scan, readConf, expectedSchema, filterExpressions, emptyScan); - } - - private Long getStartSnapshotId(Long startTimestamp) { - Snapshot oldestSnapshotAfter = SnapshotUtil.oldestAncestorAfter(table, startTimestamp); - - if (oldestSnapshotAfter == null) { + private BatchScan buildIcebergBatchScan( + Schema projection, boolean ignoreResiduals, boolean withStats) { + if (shouldPinSnapshot() && snapshot == null) { return null; - } else if (oldestSnapshotAfter.timestampMillis() == startTimestamp) { - return oldestSnapshotAfter.snapshotId(); - } else { - return oldestSnapshotAfter.parentId(); - } - } - - private Long getEndSnapshotId(Long endTimestamp) { - Long endSnapshotId = null; - for (Snapshot snapshot : SnapshotUtil.currentAncestors(table)) { - if (snapshot.timestampMillis() <= endTimestamp) { - endSnapshotId = snapshot.snapshotId(); - break; - } - } - return endSnapshotId; - } - - public Scan buildMergeOnReadScan() { - Preconditions.checkArgument( - readConf.snapshotId() == null && readConf.asOfTimestamp() == null && readConf.tag() == null, - "Cannot set time travel options %s, %s, %s for row-level command scans", - SparkReadOptions.SNAPSHOT_ID, - SparkReadOptions.AS_OF_TIMESTAMP, - SparkReadOptions.TAG); - - Preconditions.checkArgument( - readConf.startSnapshotId() == null && readConf.endSnapshotId() == null, - "Cannot set incremental scan options %s and %s for row-level command scans", - SparkReadOptions.START_SNAPSHOT_ID, - SparkReadOptions.END_SNAPSHOT_ID); - - Snapshot snapshot = SnapshotUtil.latestSnapshot(table, readConf.branch()); - - if (snapshot == null) { - return new SparkBatchQueryScan( - spark, - table, - null, - readConf, - projectionWithMetadataColumns(), - filterExpressions, - metricsReporter::scanReport); } - // remember the current snapshot ID for commit validation - long snapshotId = snapshot.snapshotId(); - - CaseInsensitiveStringMap adjustedOptions = - Spark3Util.setOption(SparkReadOptions.SNAPSHOT_ID, Long.toString(snapshotId), options); - SparkReadConf adjustedReadConf = - new SparkReadConf(spark, table, readConf.branch(), adjustedOptions); - - Schema expectedSchema = projectionWithMetadataColumns(); - BatchScan scan = - newBatchScan() - .useSnapshot(snapshotId) - .caseSensitive(caseSensitive) - .filter(filterExpression()) - .project(expectedSchema) - .metricsReporter(metricsReporter); + newIcebergBatchScan() + .caseSensitive(caseSensitive()) + .filter(filter()) + .project(projection) + .metricsReporter(metricsReporter()); - scan = configureSplitPlanning(scan); - - return new SparkBatchQueryScan( - spark, - table, - scan, - adjustedReadConf, - expectedSchema, - filterExpressions, - metricsReporter::scanReport); - } - - public Scan buildCopyOnWriteScan() { - Snapshot snapshot = SnapshotUtil.latestSnapshot(table, readConf.branch()); - - if (snapshot == null) { - return new SparkCopyOnWriteScan( - spark, - table, - readConf, - projectionWithMetadataColumns(), - filterExpressions, - metricsReporter::scanReport); + if (shouldPinSnapshot() || timeTravel != null) { + scan = scan.useSnapshot(snapshot.snapshotId()); } - Schema expectedSchema = projectionWithMetadataColumns(); - - BatchScan scan = - newBatchScan() - .useSnapshot(snapshot.snapshotId()) - .ignoreResiduals() - .caseSensitive(caseSensitive) - .filter(filterExpression()) - .project(expectedSchema) - .metricsReporter(metricsReporter); - - scan = configureSplitPlanning(scan); - - return new SparkCopyOnWriteScan( - spark, - table, - scan, + Preconditions.checkState( + Objects.equals(snapshot, scan.snapshot()), + "Failed to enforce scan consistency: resolved Spark table snapshot (%s) vs scan snapshot (%s)", snapshot, - readConf, - expectedSchema, - filterExpressions, - metricsReporter::scanReport); - } - - private > T configureSplitPlanning(T scan) { - T configuredScan = scan; + scan.snapshot()); - Long splitSize = readConf.splitSizeOption(); - if (splitSize != null) { - configuredScan = configuredScan.option(TableProperties.SPLIT_SIZE, String.valueOf(splitSize)); + if (ignoreResiduals) { + scan = scan.ignoreResiduals(); } - Integer splitLookback = readConf.splitLookbackOption(); - if (splitLookback != null) { - configuredScan = - configuredScan.option(TableProperties.SPLIT_LOOKBACK, String.valueOf(splitLookback)); + if (withStats) { + scan = scan.includeColumnStats(); } - Long splitOpenFileCost = readConf.splitOpenFileCostOption(); - if (splitOpenFileCost != null) { - configuredScan = - configuredScan.option( - TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(splitOpenFileCost)); - } + return configureSplitPlanning(scan); + } - if (null != limit) { - configuredScan = configuredScan.minRowsRequested(limit.longValue()); + private BatchScan newIcebergBatchScan() { + if (readConf().distributedPlanningEnabled()) { + return new SparkDistributedDataScan(spark(), table(), readConf()); + } else { + return table().newBatchScan(); } - - return configuredScan; } - @Override - public Statistics estimateStatistics() { - return ((SupportsReportStatistics) build()).estimateStatistics(); + private boolean shouldPinSnapshot() { + return isMainTable() || isMetadataTableWithTimeTravel(); } - @Override - public StructType readSchema() { - return build().readSchema(); + private boolean isMainTable() { + return table() instanceof BaseTable; } - private BatchScan newBatchScan() { - if (readConf.distributedPlanningEnabled()) { - return new SparkDistributedDataScan(spark, table, readConf); + private boolean isMetadataTableWithTimeTravel() { + if (table() instanceof BaseMetadataTable metadataTable) { + return metadataTable.supportsTimeTravel(); } else { - return table.newBatchScan(); + return false; } } - - @Override - public boolean pushLimit(int pushedLimit) { - this.limit = pushedLimit; - return true; - } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java index 435c2cbd15b5..685b3530370e 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.spark.source; -import java.util.Collections; import java.util.List; import java.util.Objects; import org.apache.iceberg.ScanTask; @@ -29,10 +28,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.ScanTaskSetManager; import org.apache.iceberg.spark.SparkReadConf; -import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.util.TableScanUtil; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.connector.read.Statistics; class SparkStagedScan extends SparkScan { @@ -46,23 +43,17 @@ class SparkStagedScan extends SparkScan { SparkStagedScan( SparkSession spark, Table table, + Schema schema, Schema projection, String taskSetId, SparkReadConf readConf) { - super(spark, table, readConf, projection, ImmutableList.of(), null); + super(spark, table, schema, readConf, projection, ImmutableList.of(), null); this.taskSetId = taskSetId; this.splitSize = readConf.splitSize(); this.splitLookback = readConf.splitLookback(); this.openFileCost = readConf.splitOpenFileCost(); } - @Override - public Statistics estimateStatistics() { - long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum(); - long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount); - return new Stats(sizeInBytes, rowsCount, Collections.emptyMap()); - } - @Override protected List> taskGroups() { if (taskGroups == null) { @@ -91,6 +82,7 @@ public boolean equals(Object other) { SparkStagedScan that = (SparkStagedScan) other; return table().name().equals(that.table().name()) + && Objects.equals(table().uuid(), that.table().uuid()) && Objects.equals(taskSetId, that.taskSetId) && readSchema().equals(that.readSchema()) && splitSize == that.splitSize diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index 7164c53a3d98..54ea35b8fd84 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -18,78 +18,27 @@ */ package org.apache.iceberg.spark.source; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.SparkReadConf; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.read.Scan; -import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -class SparkStagedScanBuilder implements ScanBuilder, SupportsPushDownRequiredColumns { +class SparkStagedScanBuilder extends BaseSparkScanBuilder + implements SupportsPushDownRequiredColumns { - private final SparkSession spark; - private final Table table; private final String taskSetId; - private final SparkReadConf readConf; - private final List metaColumns = Lists.newArrayList(); - - private Schema schema; SparkStagedScanBuilder( SparkSession spark, Table table, String taskSetId, CaseInsensitiveStringMap options) { - this.spark = spark; - this.table = table; + super(spark, table, table.schema(), options); this.taskSetId = taskSetId; - this.readConf = new SparkReadConf(spark, table, options); - this.schema = table.schema(); } @Override public Scan build() { - return new SparkStagedScan(spark, table, schemaWithMetadataColumns(), taskSetId, readConf); - } - - @Override - public void pruneColumns(StructType requestedSchema) { - StructType requestedProjection = removeMetaColumns(requestedSchema); - this.schema = SparkSchemaUtil.prune(schema, requestedProjection); - - Stream.of(requestedSchema.fields()) - .map(StructField::name) - .filter(MetadataColumns::isMetadataColumn) - .distinct() - .forEach(metaColumns::add); - } - - private StructType removeMetaColumns(StructType structType) { - return new StructType( - Stream.of(structType.fields()) - .filter(field -> MetadataColumns.nonMetadataColumn(field.name())) - .toArray(StructField[]::new)); - } - - private Schema schemaWithMetadataColumns() { - // metadata columns - List fields = - metaColumns.stream() - .distinct() - .map(name -> MetadataColumns.metadataColumn(table, name)) - .collect(Collectors.toList()); - Schema meta = new Schema(fields); - - // schema of rows returned by readers - return TypeUtil.join(schema, meta); + Schema projection = projectionWithMetadataColumns(); + return new SparkStagedScan(spark(), table(), schema(), projection, taskSetId, readConf()); } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 24915a1bfc47..3c162d24aab0 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -18,27 +18,20 @@ */ package org.apache.iceberg.spark.source; -import static org.apache.iceberg.TableProperties.CURRENT_SNAPSHOT_ID; -import static org.apache.iceberg.TableProperties.FORMAT_VERSION; - import java.io.IOException; -import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; -import org.apache.iceberg.BaseMetadataTable; -import org.apache.iceberg.BaseTable; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFiles; import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; -import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableScan; -import org.apache.iceberg.TableUtil; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; @@ -48,59 +41,42 @@ import org.apache.iceberg.expressions.StrictMetricsEvaluator; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.CommitMetadata; -import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkReadOptions; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.spark.SparkV2Filters; +import org.apache.iceberg.spark.TimeTravel; +import org.apache.iceberg.spark.TimeTravel.AsOfTimestamp; +import org.apache.iceberg.spark.TimeTravel.AsOfVersion; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SnapshotUtil; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.connector.catalog.MetadataColumn; import org.apache.spark.sql.connector.catalog.SupportsDeleteV2; -import org.apache.spark.sql.connector.catalog.SupportsMetadataColumns; import org.apache.spark.sql.connector.catalog.SupportsRead; import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations; import org.apache.spark.sql.connector.catalog.SupportsWrite; import org.apache.spark.sql.connector.catalog.TableCapability; -import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.connector.expressions.filter.Predicate; import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.connector.write.LogicalWriteInfo; import org.apache.spark.sql.connector.write.RowLevelOperationBuilder; import org.apache.spark.sql.connector.write.RowLevelOperationInfo; import org.apache.spark.sql.connector.write.WriteBuilder; -import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class SparkTable - implements org.apache.spark.sql.connector.catalog.Table, - SupportsRead, - SupportsWrite, - SupportsDeleteV2, - SupportsRowLevelOperations, - SupportsMetadataColumns { +/** + * The main Spark table implementation that supports reads, writes, and row-level operations. + * + *

Note the table state (e.g. schema, snapshot) is pinned upon loading and must not change. + */ +public class SparkTable extends BaseSparkTable + implements SupportsRead, SupportsWrite, SupportsDeleteV2, SupportsRowLevelOperations { private static final Logger LOG = LoggerFactory.getLogger(SparkTable.class); - private static final Set RESERVED_PROPERTIES = - ImmutableSet.of( - "provider", - "format", - CURRENT_SNAPSHOT_ID, - "location", - FORMAT_VERSION, - "sort-order", - "identifier-fields"); private static final Set CAPABILITIES = ImmutableSet.of( TableCapability.AUTOMATIC_SCHEMA_EVOLUTION, @@ -116,151 +92,59 @@ public class SparkTable .add(TableCapability.ACCEPT_ANY_SCHEMA) .build(); - private final Table icebergTable; - private final Long snapshotId; - private final boolean refreshEagerly; + private final Schema schema; // effective schema (not necessarily current table schema) + private final Snapshot snapshot; // always set unless table is empty + private final String branch; // set if table is loaded for specific branch + private final TimeTravel timeTravel; // set if table is loaded for time travel private final Set capabilities; - private final boolean isTableRewrite; - private String branch; - private StructType lazyTableSchema = null; - private SparkSession lazySpark = null; - - public SparkTable(Table icebergTable, boolean refreshEagerly) { - this(icebergTable, (Long) null, refreshEagerly); - } - - public SparkTable(Table icebergTable, String branch, boolean refreshEagerly) { - this(icebergTable, refreshEagerly); - this.branch = branch; - ValidationException.check( - branch == null - || SnapshotRef.MAIN_BRANCH.equals(branch) - || icebergTable.snapshot(branch) != null, - "Cannot use branch (does not exist): %s", - branch); - } - public SparkTable(Table icebergTable, Long snapshotId, boolean refreshEagerly) { - this(icebergTable, snapshotId, refreshEagerly, false); + public SparkTable(Table table) { + this(table, null /* main branch */); } - public SparkTable( - Table icebergTable, Long snapshotId, boolean refreshEagerly, boolean isTableRewrite) { - this.icebergTable = icebergTable; - this.snapshotId = snapshotId; - this.refreshEagerly = refreshEagerly; - - boolean acceptAnySchema = - PropertyUtil.propertyAsBoolean( - icebergTable.properties(), - TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA, - TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA_DEFAULT); - this.capabilities = acceptAnySchema ? CAPABILITIES_WITH_ACCEPT_ANY_SCHEMA : CAPABILITIES; - this.isTableRewrite = isTableRewrite; + private SparkTable(Table table, String branch) { + this( + table, + table.schema(), + determineLatestSnapshot(table, branch), + branch, + null /* no time travel */); } - private SparkSession sparkSession() { - if (lazySpark == null) { - this.lazySpark = SparkSession.active(); - } - - return lazySpark; + private SparkTable(Table table, long snapshotId, TimeTravel timeTravel) { + this( + table, + SnapshotUtil.schemaFor(table, snapshotId), + table.snapshot(snapshotId), + null /* main branch */, + timeTravel); } - public Table table() { - return icebergTable; + private SparkTable( + Table table, Schema schema, Snapshot snapshot, String branch, TimeTravel timeTravel) { + super(table, schema); + this.schema = schema; + this.snapshot = snapshot; + this.branch = branch; + this.timeTravel = timeTravel; + this.capabilities = acceptAnySchema(table) ? CAPABILITIES_WITH_ACCEPT_ANY_SCHEMA : CAPABILITIES; } - @Override - public String name() { - return icebergTable.toString(); + public SparkTable copyWithBranch(String newBranch) { + return new SparkTable(table(), newBranch); } public Long snapshotId() { - return snapshotId; + return snapshot != null ? snapshot.snapshotId() : null; } public String branch() { return branch; } - public SparkTable copyWithSnapshotId(long newSnapshotId) { - return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); - } - - public SparkTable copyWithBranch(String targetBranch) { - return new SparkTable(icebergTable, targetBranch, refreshEagerly); - } - - private Schema snapshotSchema() { - if (icebergTable instanceof BaseMetadataTable) { - return icebergTable.schema(); - } else if (branch != null) { - return addLineageIfRequired(SnapshotUtil.schemaFor(icebergTable, branch)); - } else { - return addLineageIfRequired(SnapshotUtil.schemaFor(icebergTable, snapshotId, null)); - } - } - - private Schema addLineageIfRequired(Schema schema) { - if (TableUtil.supportsRowLineage(icebergTable) && isTableRewrite) { - return MetadataColumns.schemaWithRowLineage(schema); - } - - return schema; - } - @Override - public StructType schema() { - if (lazyTableSchema == null) { - this.lazyTableSchema = SparkSchemaUtil.convert(snapshotSchema()); - } - - return lazyTableSchema; - } - - @Override - public Transform[] partitioning() { - return Spark3Util.toTransforms(icebergTable.spec()); - } - - @Override - public Map properties() { - ImmutableMap.Builder propsBuilder = ImmutableMap.builder(); - - String fileFormat = - icebergTable - .properties() - .getOrDefault( - TableProperties.DEFAULT_FILE_FORMAT, TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); - propsBuilder.put("format", "iceberg/" + fileFormat); - propsBuilder.put("provider", "iceberg"); - String currentSnapshotId = - icebergTable.currentSnapshot() != null - ? String.valueOf(icebergTable.currentSnapshot().snapshotId()) - : "none"; - propsBuilder.put(CURRENT_SNAPSHOT_ID, currentSnapshotId); - propsBuilder.put("location", icebergTable.location()); - - if (icebergTable instanceof BaseTable) { - TableOperations ops = ((BaseTable) icebergTable).operations(); - propsBuilder.put(FORMAT_VERSION, String.valueOf(ops.current().formatVersion())); - } - - if (!icebergTable.sortOrder().isUnsorted()) { - propsBuilder.put("sort-order", Spark3Util.describe(icebergTable.sortOrder())); - } - - Set identifierFields = icebergTable.schema().identifierFieldNames(); - if (!identifierFields.isEmpty()) { - propsBuilder.put("identifier-fields", "[" + String.join(",", identifierFields) + "]"); - } - - icebergTable.properties().entrySet().stream() - .filter(entry -> !RESERVED_PROPERTIES.contains(entry.getKey())) - .forEach(propsBuilder::put); - - return propsBuilder.build(); + public String version() { + return String.format("branch_%s_snapshot_%s", branch, snapshotId()); } @Override @@ -268,52 +152,26 @@ public Set capabilities() { return capabilities; } - @Override - public MetadataColumn[] metadataColumns() { - List cols = Lists.newArrayList(); - - cols.add(SparkMetadataColumns.SPEC_ID); - cols.add(SparkMetadataColumns.partition(icebergTable)); - cols.add(SparkMetadataColumns.FILE_PATH); - cols.add(SparkMetadataColumns.ROW_POSITION); - cols.add(SparkMetadataColumns.IS_DELETED); - - if (TableUtil.supportsRowLineage(icebergTable)) { - cols.add(SparkMetadataColumns.ROW_ID); - cols.add(SparkMetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER); - } - - return cols.toArray(SparkMetadataColumn[]::new); - } - @Override public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { - if (refreshEagerly) { - icebergTable.refresh(); - } - - CaseInsensitiveStringMap scanOptions = - branch != null ? options : addSnapshotId(options, snapshotId); - return new SparkScanBuilder( - sparkSession(), icebergTable, branch, snapshotSchema(), scanOptions); + return new SparkScanBuilder(spark(), table(), schema, snapshot, branch, timeTravel, options); } @Override public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { - Preconditions.checkArgument( - snapshotId == null, "Cannot write to table at a specific snapshot: %s", snapshotId); - return new SparkWriteBuilder(sparkSession(), icebergTable, branch, info); + Preconditions.checkArgument(timeTravel == null, "Cannot write to table with time travel"); + return new SparkWriteBuilder(spark(), table(), branch, info); } @Override public RowLevelOperationBuilder newRowLevelOperationBuilder(RowLevelOperationInfo info) { - return new SparkRowLevelOperationBuilder(sparkSession(), icebergTable, branch, info); + Preconditions.checkArgument(timeTravel == null, "Cannot modify table with time travel"); + return new SparkRowLevelOperationBuilder(spark(), table(), snapshot, branch, info); } @Override public boolean canDeleteWhere(Predicate[] predicates) { - Preconditions.checkArgument( - snapshotId == null, "Cannot delete from table at a specific snapshot: %s", snapshotId); + Preconditions.checkArgument(timeTravel == null, "Cannot delete from table with time travel"); Expression deleteExpr = Expressions.alwaysTrue(); @@ -331,7 +189,7 @@ public boolean canDeleteWhere(Predicate[] predicates) { // a metadata delete is possible iff matching files can be deleted entirely private boolean canDeleteUsingMetadata(Expression deleteExpr) { - boolean caseSensitive = SparkUtil.caseSensitive(sparkSession()); + boolean caseSensitive = SparkUtil.caseSensitive(spark()); if (ExpressionUtil.selectsPartitions(deleteExpr, table(), caseSensitive)) { return true; @@ -345,15 +203,13 @@ private boolean canDeleteUsingMetadata(Expression deleteExpr) { .includeColumnStats() .ignoreResiduals(); - if (branch != null) { - scan = scan.useRef(branch); + if (snapshot != null) { + scan = scan.useSnapshot(snapshot.snapshotId()); } try (CloseableIterable tasks = scan.planFiles()) { Map evaluators = Maps.newHashMap(); - StrictMetricsEvaluator metricsEvaluator = - new StrictMetricsEvaluator(SnapshotUtil.schemaFor(table(), branch), deleteExpr); - + StrictMetricsEvaluator metricsEvaluator = new StrictMetricsEvaluator(schema, deleteExpr); return Iterables.all( tasks, task -> { @@ -384,15 +240,11 @@ public void deleteWhere(Predicate[] predicates) { } DeleteFiles deleteFiles = - icebergTable + table() .newDelete() - .set("spark.app.id", sparkSession().sparkContext().applicationId()) + .set("spark.app.id", spark().sparkContext().applicationId()) .deleteFromRowFilter(deleteExpr); - if (SparkTableUtil.wapEnabled(table())) { - branch = SparkTableUtil.determineWriteBranch(sparkSession(), icebergTable, branch); - } - if (branch != null) { deleteFiles.toBranch(branch); } @@ -404,11 +256,6 @@ public void deleteWhere(Predicate[] predicates) { deleteFiles.commit(); } - @Override - public String toString() { - return icebergTable.toString(); - } - @Override public boolean equals(Object other) { if (this == other) { @@ -417,37 +264,78 @@ public boolean equals(Object other) { return false; } - // use only name in order to correctly invalidate Spark cache SparkTable that = (SparkTable) other; - return icebergTable.name().equals(that.icebergTable.name()); + return table().name().equals(that.table().name()) + && Objects.equals(table().uuid(), that.table().uuid()) + && schema.schemaId() == that.schema.schemaId() + && Objects.equals(snapshotId(), that.snapshotId()) + && Objects.equals(branch, that.branch) + && Objects.equals(timeTravel, that.timeTravel); } @Override public int hashCode() { - // use only name in order to correctly invalidate Spark cache - return icebergTable.name().hashCode(); + return Objects.hash( + table().name(), table().uuid(), schema.schemaId(), snapshotId(), branch, timeTravel); } - private static CaseInsensitiveStringMap addSnapshotId( - CaseInsensitiveStringMap options, Long snapshotId) { - if (snapshotId != null) { - String snapshotIdFromOptions = options.get(SparkReadOptions.SNAPSHOT_ID); - String value = snapshotId.toString(); + public static SparkTable create(Table table, String branch) { + ValidationException.check( + branch == null || SnapshotRef.MAIN_BRANCH.equals(branch) || table.snapshot(branch) != null, + "Cannot use branch (does not exist): %s", + branch); + return new SparkTable(table, branch); + } + + public static SparkTable create(Table table, TimeTravel timeTravel) { + if (timeTravel == null) { + return new SparkTable(table); + } else if (timeTravel instanceof AsOfVersion asOfVersion) { + return createWithVersion(table, asOfVersion); + } else if (timeTravel instanceof AsOfTimestamp asOfTimestamp) { + return createWithTimestamp(table, asOfTimestamp); + } else { + throw new IllegalArgumentException("Unknown time travel: " + timeTravel); + } + } + + private static SparkTable createWithVersion(Table table, AsOfVersion timeTravel) { + if (timeTravel.isSnapshotId()) { + return new SparkTable(table, Long.parseLong(timeTravel.version()), timeTravel); + } else { + SnapshotRef ref = table.refs().get(timeTravel.version()); Preconditions.checkArgument( - snapshotIdFromOptions == null || snapshotIdFromOptions.equals(value), - "Cannot override snapshot ID more than once: %s", - snapshotIdFromOptions); - - Map scanOptions = Maps.newHashMap(); - scanOptions.putAll(options.asCaseSensitiveMap()); - scanOptions.put(SparkReadOptions.SNAPSHOT_ID, value); - scanOptions.remove(SparkReadOptions.AS_OF_TIMESTAMP); - scanOptions.remove(SparkReadOptions.BRANCH); - scanOptions.remove(SparkReadOptions.TAG); - - return new CaseInsensitiveStringMap(scanOptions); + ref != null, + "Cannot find matching snapshot ID or reference name for version %s", + timeTravel.version()); + if (ref.isBranch()) { + return new SparkTable(table, timeTravel.version()); + } else { + return new SparkTable(table, ref.snapshotId(), timeTravel); + } } + } + + // Iceberg uses milliseconds for snapshot timestamps + private static SparkTable createWithTimestamp(Table table, AsOfTimestamp timeTravel) { + long timestampMillis = timeTravel.timestampMillis(); + long snapshotId = SnapshotUtil.snapshotIdAsOfTime(table, timestampMillis); + return new SparkTable(table, snapshotId, timeTravel); + } - return options; + private static boolean acceptAnySchema(Table table) { + return PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA, + TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA_DEFAULT); + } + + // returns latest snapshot for branch or current snapshot if branch is yet to be created + private static Snapshot determineLatestSnapshot(Table table, String branch) { + if (branch != null && table.refs().containsKey(branch)) { + return SnapshotUtil.latestSnapshot(table, branch); + } else { + return table.currentSnapshot(); + } } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index e0a05ff11a7e..06856eee3b28 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.util.Arrays; -import java.util.List; import java.util.Locale; import java.util.Map; import java.util.function.Function; @@ -114,6 +113,7 @@ abstract class SparkWrite extends BaseSparkWrite implements Write, RequiresDistr SparkWrite( SparkSession spark, Table table, + String branch, SparkWriteConf writeConf, LogicalWriteInfo writeInfo, String applicationId, @@ -128,7 +128,7 @@ abstract class SparkWrite extends BaseSparkWrite implements Write, RequiresDistr this.applicationId = applicationId; this.wapEnabled = writeConf.wapEnabled(); this.wapId = writeConf.wapId(); - this.branch = writeConf.branch(); + this.branch = branch; this.targetFileSize = writeConf.targetDataFileSize(); this.writeSchema = writeSchema; this.dsSchema = dsSchema; @@ -428,19 +428,6 @@ private DeleteFileSet danglingDVs() { } } - private Expression conflictDetectionFilter() { - // the list of filter expressions may be empty but is never null - List scanFilterExpressions = scan.filterExpressions(); - - Expression filter = Expressions.alwaysTrue(); - - for (Expression expr : scanFilterExpressions) { - filter = Expressions.and(filter, expr); - } - - return filter; - } - @Override public void commit(WriterCommitMessage[] messages) { commit(messages, null); @@ -496,7 +483,7 @@ private void commitWithSerializableIsolation( overwriteFiles.validateFromSnapshot(scanSnapshotId); } - Expression conflictDetectionFilter = conflictDetectionFilter(); + Expression conflictDetectionFilter = scan.filter(); overwriteFiles.conflictDetectionFilter(conflictDetectionFilter); overwriteFiles.validateNoConflictingData(); overwriteFiles.validateNoConflictingDeletes(); @@ -522,7 +509,7 @@ private void commitWithSnapshotIsolation( overwriteFiles.validateFromSnapshot(scanSnapshotId); } - Expression conflictDetectionFilter = conflictDetectionFilter(); + Expression conflictDetectionFilter = scan.filter(); overwriteFiles.conflictDetectionFilter(conflictDetectionFilter); overwriteFiles.validateNoConflictingDeletes(); diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java index 182e56a861ce..0d0d6146a4a9 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java @@ -29,9 +29,14 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.SparkFilters; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.spark.SparkWriteConf; import org.apache.iceberg.spark.SparkWriteRequirements; +import org.apache.iceberg.spark.source.SparkWriteBuilder.Mode.Append; +import org.apache.iceberg.spark.source.SparkWriteBuilder.Mode.CopyOnWriteOperation; +import org.apache.iceberg.spark.source.SparkWriteBuilder.Mode.DynamicOverwrite; +import org.apache.iceberg.spark.source.SparkWriteBuilder.Mode.OverwriteByFilter; import org.apache.iceberg.types.TypeUtil; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.read.Scan; @@ -44,115 +49,116 @@ import org.apache.spark.sql.connector.write.WriteBuilder; import org.apache.spark.sql.connector.write.streaming.StreamingWrite; import org.apache.spark.sql.sources.Filter; -import org.apache.spark.sql.types.LongType$; +import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; class SparkWriteBuilder implements WriteBuilder, SupportsDynamicOverwrite, SupportsOverwrite { + private final SparkSession spark; private final Table table; + private final String branch; private final SparkWriteConf writeConf; - private final LogicalWriteInfo writeInfo; - private final StructType dsSchema; - private final String overwriteMode; - private boolean overwriteDynamic = false; - private boolean overwriteByFilter = false; - private Expression overwriteExpr = null; - private boolean overwriteFiles = false; - private SparkCopyOnWriteScan copyOnWriteScan = null; - private Command copyOnWriteCommand = null; - private IsolationLevel copyOnWriteIsolationLevel = null; + private final LogicalWriteInfo info; + private final boolean caseSensitive; + private final boolean checkNullability; + private final boolean checkOrdering; + private final boolean mergeSchema; + private Mode mode = null; SparkWriteBuilder(SparkSession spark, Table table, String branch, LogicalWriteInfo info) { this.spark = spark; this.table = table; - this.writeConf = new SparkWriteConf(spark, table, branch, info.options()); - this.writeInfo = info; - this.dsSchema = info.schema(); - this.overwriteMode = writeConf.overwriteMode(); + this.branch = branch; + this.writeConf = new SparkWriteConf(spark, table, info.options()); + this.info = info; + this.caseSensitive = writeConf.caseSensitive(); + this.checkNullability = writeConf.checkNullability(); + this.checkOrdering = writeConf.checkOrdering(); + this.mergeSchema = writeConf.mergeSchema(); + SparkTableUtil.validateWriteBranch(spark, table, branch, info.options()); + } + + private Mode mode() { + return mode != null ? mode : new Append(); } public WriteBuilder overwriteFiles(Scan scan, Command command, IsolationLevel isolationLevel) { - Preconditions.checkState(!overwriteByFilter, "Cannot overwrite individual files and by filter"); - Preconditions.checkState( - !overwriteDynamic, "Cannot overwrite individual files and dynamically"); - - this.overwriteFiles = true; - this.copyOnWriteScan = (SparkCopyOnWriteScan) scan; - this.copyOnWriteCommand = command; - this.copyOnWriteIsolationLevel = isolationLevel; + Preconditions.checkState(mode == null, "Cannot use copy-on-write with other modes"); + this.mode = new CopyOnWriteOperation((SparkCopyOnWriteScan) scan, command, isolationLevel); return this; } @Override public WriteBuilder overwriteDynamicPartitions() { - Preconditions.checkState( - !overwriteByFilter, "Cannot overwrite dynamically and by filter: %s", overwriteExpr); - Preconditions.checkState(!overwriteFiles, "Cannot overwrite individual files and dynamically"); - - this.overwriteDynamic = true; + Preconditions.checkState(mode == null, "Cannot use dynamic overwrite with other modes"); + this.mode = new DynamicOverwrite(); return this; } @Override public WriteBuilder overwrite(Filter[] filters) { - Preconditions.checkState( - !overwriteFiles, "Cannot overwrite individual files and using filters"); - - this.overwriteExpr = SparkFilters.convert(filters); - if (overwriteExpr == Expressions.alwaysTrue() && "dynamic".equals(overwriteMode)) { + Preconditions.checkState(mode == null, "Cannot use overwrite by filter with other modes"); + Expression expr = SparkFilters.convert(filters); + String overwriteMode = writeConf.overwriteMode(); + if (expr == Expressions.alwaysTrue() && "dynamic".equals(overwriteMode)) { // use the write option to override truncating the table. use dynamic overwrite instead. - this.overwriteDynamic = true; + this.mode = new DynamicOverwrite(); } else { - Preconditions.checkState( - !overwriteDynamic, "Cannot overwrite dynamically and by filter: %s", overwriteExpr); - this.overwriteByFilter = true; + this.mode = new OverwriteByFilter(expr); } return this; } - @Override - public Write build() { - // The write schema should only include row lineage in the output if it's an overwrite - // operation or if it's a compaction. - // In any other case, only null row IDs and sequence numbers would be produced which - // means the row lineage columns can be excluded from the output files - boolean writeRequiresRowLineage = TableUtil.supportsRowLineage(table) && overwriteFiles; - boolean writeAlreadyIncludesLineage = - dsSchema.exists(field -> field.name().equals(MetadataColumns.ROW_ID.name())); - StructType sparkWriteSchema = dsSchema; - if (writeRequiresRowLineage && !writeAlreadyIncludesLineage) { - sparkWriteSchema = sparkWriteSchema.add(MetadataColumns.ROW_ID.name(), LongType$.MODULE$); - sparkWriteSchema = - sparkWriteSchema.add( - MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name(), LongType$.MODULE$); - } + private boolean writeNeedsRowLineage() { + return TableUtil.supportsRowLineage(table) && mode instanceof CopyOnWriteOperation; + } - Schema writeSchema = - validateOrMergeWriteSchema(table, sparkWriteSchema, writeConf, writeRequiresRowLineage); + private boolean writeIncludesRowLineage() { + return info.metadataSchema() + .map(schema -> schema.exists(field -> field.name().equals(MetadataColumns.ROW_ID.name()))) + .orElse(false); + } - SparkUtil.validatePartitionTransforms(table.spec()); + private StructType sparkWriteSchema() { + if (writeIncludesRowLineage()) { + StructType schema = info.schema(); + StructType metaSchema = info.metadataSchema().get(); + StructField rowId = metaSchema.apply(MetadataColumns.ROW_ID.name()); + schema = schema.add(rowId); + StructField seq = metaSchema.apply(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name()); + schema = schema.add(seq); + return schema; + } else { + return info.schema(); + } + } - // Get application id + @Override + public Write build() { + validateRowLineage(); + Schema writeSchema = mergeSchema ? mergeAndValidateWriteSchema() : validateWriteSchema(); + SparkUtil.validatePartitionTransforms(table.spec()); String appId = spark.sparkContext().applicationId(); - return new SparkWrite( spark, table, + branch, writeConf, - writeInfo, + info, appId, writeSchema, - sparkWriteSchema, + sparkWriteSchema(), writeRequirements()) { @Override public BatchWrite toBatch() { - if (overwriteByFilter) { - return asOverwriteByFilter(overwriteExpr); - } else if (overwriteDynamic) { + Mode currentMode = mode(); + if (currentMode instanceof OverwriteByFilter overwrite) { + return asOverwriteByFilter(overwrite.expr()); + } else if (currentMode instanceof DynamicOverwrite) { return asDynamicOverwrite(); - } else if (overwriteFiles) { - return asCopyOnWriteOperation(copyOnWriteScan, copyOnWriteIsolationLevel); + } else if (currentMode instanceof CopyOnWriteOperation cow) { + return asCopyOnWriteOperation(cow.scan(), cow.isolationLevel()); } else { return asBatchAppend(); } @@ -160,66 +166,76 @@ public BatchWrite toBatch() { @Override public StreamingWrite toStreaming() { - Preconditions.checkState( - !overwriteDynamic, "Unsupported streaming operation: dynamic partition overwrite"); - Preconditions.checkState( - !overwriteByFilter || overwriteExpr == Expressions.alwaysTrue(), - "Unsupported streaming operation: overwrite by filter: %s", - overwriteExpr); - - if (overwriteByFilter) { + Mode currentMode = mode(); + if (currentMode instanceof Append) { + return asStreamingAppend(); + } else if (currentMode instanceof OverwriteByFilter overwrite) { + Preconditions.checkState( + overwrite.expr() == Expressions.alwaysTrue(), + "Unsupported streaming overwrite filter: " + overwrite.expr()); return asStreamingOverwrite(); } else { - return asStreamingAppend(); + throw new IllegalStateException("Unsupported streaming write mode: " + currentMode); } } }; } private SparkWriteRequirements writeRequirements() { - if (overwriteFiles) { - return writeConf.copyOnWriteRequirements(copyOnWriteCommand); + if (mode instanceof CopyOnWriteOperation cow) { + return writeConf.copyOnWriteRequirements(cow.command()); } else { return writeConf.writeRequirements(); } } - private static Schema validateOrMergeWriteSchema( - Table table, StructType dsSchema, SparkWriteConf writeConf, boolean writeIncludesRowLineage) { - Schema writeSchema; - boolean caseSensitive = writeConf.caseSensitive(); - if (writeConf.mergeSchema()) { - // convert the dataset schema and assign fresh ids for new fields - Schema newSchema = - SparkSchemaUtil.convertWithFreshIds(table.schema(), dsSchema, caseSensitive); - - // update the table to get final id assignments and validate the changes - UpdateSchema update = - table.updateSchema().caseSensitive(caseSensitive).unionByNameWith(newSchema); - Schema mergedSchema = update.apply(); - if (writeIncludesRowLineage) { - mergedSchema = - TypeUtil.join(mergedSchema, MetadataColumns.schemaWithRowLineage(table.schema())); - } + private void validateRowLineage() { + Preconditions.checkArgument( + writeIncludesRowLineage() || !writeNeedsRowLineage(), + "Row lineage information is missing for write in %s mode", + mode()); + } - // reconvert the dsSchema without assignment to use the ids assigned by UpdateSchema - writeSchema = SparkSchemaUtil.convert(mergedSchema, dsSchema, caseSensitive); + private Schema validateWriteSchema() { + Schema writeSchema = SparkSchemaUtil.convert(table.schema(), info.schema(), caseSensitive); + TypeUtil.validateWriteSchema(table.schema(), writeSchema, checkNullability, checkOrdering); + return addRowLineageIfNeeded(writeSchema); + } - TypeUtil.validateWriteSchema( - mergedSchema, writeSchema, writeConf.checkNullability(), writeConf.checkOrdering()); + // merge schema flow: + // - convert the Spark schema and assign fresh IDs for new fields + // - update the table to get final ID assignments and validate the changes + // - reconvert the Spark schema without assignment to use the IDs assigned by UpdateSchema + // - if the validation passed, update the table schema + private Schema mergeAndValidateWriteSchema() { + Schema newSchema = + SparkSchemaUtil.convertWithFreshIds(table.schema(), info.schema(), caseSensitive); + UpdateSchema update = + table.updateSchema().caseSensitive(caseSensitive).unionByNameWith(newSchema); + Schema mergedSchema = update.apply(); + Schema writeSchema = SparkSchemaUtil.convert(mergedSchema, info.schema(), caseSensitive); + TypeUtil.validateWriteSchema(mergedSchema, writeSchema, checkNullability, checkOrdering); + update.commit(); + return addRowLineageIfNeeded(writeSchema); + } - // if the validation passed, update the table schema - update.commit(); - } else { - Schema schema = - writeIncludesRowLineage - ? MetadataColumns.schemaWithRowLineage(table.schema()) - : table.schema(); - writeSchema = SparkSchemaUtil.convert(schema, dsSchema, caseSensitive); - TypeUtil.validateWriteSchema( - table.schema(), writeSchema, writeConf.checkNullability(), writeConf.checkOrdering()); - } + private Schema addRowLineageIfNeeded(Schema schema) { + return writeNeedsRowLineage() ? MetadataColumns.schemaWithRowLineage(schema) : schema; + } + + sealed interface Mode { + // add new data + record Append() implements Mode {} + + // overwrite partitions that receive new data (determined at runtime) + record DynamicOverwrite() implements Mode {} + + // overwrite data files matching filter expression (a.k.a static overwrite) + record OverwriteByFilter(Expression expr) implements Mode {} - return writeSchema; + // copy-on-write operation (UPDATE/DELETE/MERGE) that rewrites affected files + record CopyOnWriteOperation( + SparkCopyOnWriteScan scan, Command command, IsolationLevel isolationLevel) + implements Mode {} } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java index b92c02d2b536..d78f83a51140 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java @@ -25,7 +25,7 @@ public class StagedSparkTable extends SparkTable implements StagedTable { private final Transaction transaction; public StagedSparkTable(Transaction transaction) { - super(transaction.table(), false); + super(transaction.table()); this.transaction = transaction; } diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 0d2a5c0a4daf..5a33c710b2f6 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -79,6 +79,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.spark.actions.DeleteOrphanFilesSparkAction.StringToFileURI; @@ -724,7 +725,7 @@ private List snapshotFiles(long snapshotId) { return spark .read() .format("iceberg") - .option("snapshot-id", snapshotId) + .option(SparkReadOptions.VERSION_AS_OF, snapshotId) .load(tableLocation + "#files") .select("file_path") .as(Encoders.STRING()) diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index a7702b169a60..5fee9624e340 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -24,6 +24,8 @@ import java.io.IOException; import java.math.RoundingMode; +import java.text.SimpleDateFormat; +import java.util.Date; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -66,6 +68,8 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestDataSourceOptions extends TestBaseWithCatalog { + private static final SimpleDateFormat TIMESTAMP_FORMAT = + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); private static final Configuration CONF = new Configuration(); private static final Schema SCHEMA = new Schema( @@ -252,29 +256,28 @@ public void testIncrementalScanOptions() throws IOException { spark .read() .format("iceberg") - .option("snapshot-id", snapshotIds.get(3).toString()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotIds.get(3).toString()) .option("start-snapshot-id", snapshotIds.get(3).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); + .hasMessage("Cannot use time travel in incremental scan"); // end-snapshot-id and as-of-timestamp are both configured. + long snapshotTimestamp = table.snapshot(snapshotIds.get(3)).timestampMillis(); + String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(snapshotTimestamp)); + assertThatThrownBy( () -> spark .read() .format("iceberg") - .option( - SparkReadOptions.AS_OF_TIMESTAMP, - Long.toString(table.snapshot(snapshotIds.get(3)).timestampMillis())) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) .option("end-snapshot-id", snapshotIds.get(2).toString()) .load(tableLocation) .explain()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); + .hasMessage("Cannot use time travel in incremental scan"); // only end-snapshot-id is configured. assertThatThrownBy( @@ -287,7 +290,7 @@ public void testIncrementalScanOptions() throws IOException { .explain()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot set only end-snapshot-id for incremental scans. Please, set start-snapshot-id too."); + "Cannot set only `end-snapshot-id` for incremental scans. Please, set `start-snapshot-id` too."); // test (1st snapshot, current snapshot] incremental scan. Dataset unboundedIncrementalResult = diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java index f2d7a7f7d590..16988d1e28e9 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java @@ -220,11 +220,9 @@ public void testUnpartitionedCaseInsensitiveIDFilters() { TestFilteredScan.spark.conf().set("spark.sql.caseSensitive", "false"); try { - for (int i = 0; i < 10; i += 1) { SparkScanBuilder builder = - new SparkScanBuilder(spark, TABLES.load(options.get("path")), options) - .caseSensitive(false); + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); pushFilters( builder, @@ -294,13 +292,6 @@ public void limitPushedDownToSparkScan() { scanAssert.extracting("context").extracting("minRowsRequested").isEqualTo(limit); - // verify changelog scan - assertThat(builder.buildChangelogScan()) - .extracting("scan") - .extracting("context") - .extracting("minRowsRequested") - .isEqualTo(limit); - // verify CoW scan scanAssert = assertThat(builder.buildCopyOnWriteScan()).extracting("scan"); if (LOCAL == planningMode) { @@ -310,7 +301,7 @@ public void limitPushedDownToSparkScan() { scanAssert.extracting("context").extracting("minRowsRequested").isEqualTo(limit); // verify MoR scan - scanAssert = assertThat(builder.buildMergeOnReadScan()).extracting("scan"); + scanAssert = assertThat(builder.build()).extracting("scan"); if (LOCAL == planningMode) { scanAssert = scanAssert.extracting("scan"); } @@ -351,14 +342,6 @@ public void limitPushedDownToSparkScanForMetadataTable() { .extracting("context") .extracting("minRowsRequested") .isEqualTo(limit); - - // verify MoR scan - assertThat(builder.buildMergeOnReadScan()) - .extracting("scan") - .extracting("scan") - .extracting("context") - .extracting("minRowsRequested") - .isEqualTo(limit); } @TestTemplate diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 290e73c3bd1e..09dcad125150 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -59,6 +59,7 @@ import org.apache.spark.sql.streaming.StreamingQueryException; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import scala.Option; @@ -166,6 +167,7 @@ public void testSparkStreamingWriteFailsUnknownTransform() throws IOException, T .hasMessageContaining("Cannot write using unsupported transforms: zero"); } + @Disabled("This test is temporarily disabled as Spark 4.1.0 eagerly requests metadata columns") @Test public void testSparkCanReadUnknownTransform() throws IOException { File parent = temp.resolve("avro").toFile(); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 67647925e59d..3e5cc26d9c59 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -30,11 +30,13 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; +import java.text.SimpleDateFormat; import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import java.util.Date; import java.util.List; import java.util.Map; import java.util.StringJoiner; @@ -103,6 +105,9 @@ public abstract class TestIcebergSourceTablesBase extends TestBase { + private static final SimpleDateFormat TIMESTAMP_FORMAT = + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + private static final Schema SCHEMA = new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); @@ -1430,7 +1435,7 @@ public void testPartitionsTable() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, String.valueOf(firstCommitId)) + .option(SparkReadOptions.VERSION_AS_OF, String.valueOf(firstCommitId)) .load(loadLocation(tableIdentifier, "partitions")) .orderBy("partition.id") .collectAsList(); @@ -1799,7 +1804,7 @@ public synchronized void testSnapshotReadAfterAddColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1838,6 +1843,9 @@ public synchronized void testSnapshotReadAfterDropColumn() { table.updateSchema().deleteColumn("data").commit(); long tsAfterDropColumn = waitUntilAfter(System.currentTimeMillis()); + String formattedTsBeforeDropColumn = TIMESTAMP_FORMAT.format(new Date(tsBeforeDropColumn)); + String formattedTsAfterDropColumn = TIMESTAMP_FORMAT.format(new Date(tsAfterDropColumn)); + List newRecords = Lists.newArrayList(RowFactory.create(4, "B"), RowFactory.create(5, "C")); StructType newSparkSchema = SparkSchemaUtil.convert(SCHEMA3); @@ -1868,7 +1876,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, tsBeforeDropColumn) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsBeforeDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf3.orderBy("id").collectAsList()) .as("Records should match") @@ -1881,7 +1889,7 @@ public synchronized void testSnapshotReadAfterDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, tsAfterDropColumn) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTsAfterDropColumn) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -1964,7 +1972,7 @@ public synchronized void testSnapshotReadAfterAddAndDropColumn() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) + .option(SparkReadOptions.VERSION_AS_OF, snapshotBeforeAddColumn.snapshotId()) .load(loadLocation(tableIdentifier)); assertThat(resultDf4.orderBy("id").collectAsList()) .as("Records should match") @@ -2328,7 +2336,7 @@ public void testSessionConfigSupport() { withSQLConf( // set read option through session configuration - ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), + ImmutableMap.of("spark.datasource.iceberg.versionAsOf", String.valueOf(s1)), () -> { Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java index 11865db7fce5..416f57e5142a 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java @@ -26,6 +26,8 @@ import java.net.InetAddress; import java.nio.file.Path; +import java.text.SimpleDateFormat; +import java.util.Date; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -43,6 +45,7 @@ import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Types; +import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; @@ -57,6 +60,9 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSnapshotSelection { + private static final SimpleDateFormat TIMESTAMP_FORMAT = + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + @Parameters(name = "properties = {0}") public static Object[] parameters() { return new Object[][] { @@ -140,7 +146,11 @@ public void testSnapshotSelectionById() { Snapshot currentSnapshot = table.currentSnapshot(); Long parentSnapshotId = currentSnapshot.parentId(); Dataset previousSnapshotResult = - spark.read().format("iceberg").option("snapshot-id", parentSnapshotId).load(tableLocation); + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VERSION_AS_OF, parentSnapshotId) + .load(tableLocation); List previousSnapshotRecords = previousSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); assertThat(previousSnapshotRecords) @@ -165,6 +175,7 @@ public void testSnapshotSelectionByTimestamp() { // remember the time when the first snapshot was valid long firstSnapshotTimestamp = System.currentTimeMillis(); + String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(firstSnapshotTimestamp)); // produce the second snapshot List secondBatchRecords = @@ -191,7 +202,7 @@ public void testSnapshotSelectionByTimestamp() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, firstSnapshotTimestamp) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) .load(tableLocation); List previousSnapshotRecords = previousSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); @@ -208,9 +219,13 @@ public void testSnapshotSelectionByInvalidSnapshotId() { PartitionSpec spec = PartitionSpec.unpartitioned(); tables.create(SCHEMA, spec, properties, tableLocation); - Dataset df = spark.read().format("iceberg").option("snapshot-id", -10).load(tableLocation); - - assertThatThrownBy(df::collectAsList) + assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VERSION_AS_OF, -10) + .load(tableLocation)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find snapshot with ID -10"); } @@ -218,6 +233,7 @@ public void testSnapshotSelectionByInvalidSnapshotId() { @TestTemplate public void testSnapshotSelectionByInvalidTimestamp() { long timestamp = System.currentTimeMillis(); + String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(timestamp)); String tableLocation = temp.resolve("iceberg-table").toFile().toString(); HadoopTables tables = new HadoopTables(CONF); @@ -229,7 +245,7 @@ public void testSnapshotSelectionByInvalidTimestamp() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) .load(tableLocation)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot find a snapshot older than"); @@ -250,6 +266,7 @@ public void testSnapshotSelectionBySnapshotIdAndTimestamp() { firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); long timestamp = System.currentTimeMillis(); + String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(timestamp)); long snapshotId = table.currentSnapshot().snapshotId(); assertThatThrownBy( @@ -257,14 +274,11 @@ public void testSnapshotSelectionBySnapshotIdAndTimestamp() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) - .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .option(SparkReadOptions.VERSION_AS_OF, snapshotId) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) .load(tableLocation)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Can specify only one of snapshot-id") - .hasMessageContaining("as-of-timestamp") - .hasMessageContaining("branch") - .hasMessageContaining("tag"); + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot specify both version and timestamp"); } @TestTemplate @@ -293,7 +307,11 @@ public void testSnapshotSelectionByTag() { // verify records in the current snapshot by tag Dataset currentSnapshotResult = - spark.read().format("iceberg").option("tag", "tag").load(tableLocation); + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VERSION_AS_OF, "tag") + .load(tableLocation); List currentSnapshotRecords = currentSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); List expectedRecords = Lists.newArrayList(); @@ -362,12 +380,12 @@ public void testSnapshotSelectionByBranchAndTagFails() { spark .read() .format("iceberg") - .option(SparkReadOptions.TAG, "tag") + .option(SparkReadOptions.VERSION_AS_OF, "tag") .option(SparkReadOptions.BRANCH, "branch") .load(tableLocation) .show()) .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Can specify only one of snapshot-id"); + .hasMessageStartingWith("Cannot time travel in branch"); } @TestTemplate @@ -385,6 +403,7 @@ public void testSnapshotSelectionByTimestampAndBranchOrTagFails() { firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); long timestamp = System.currentTimeMillis(); + String formattedTimestamp = TIMESTAMP_FORMAT.format(new Date(timestamp)); table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); @@ -393,24 +412,24 @@ public void testSnapshotSelectionByTimestampAndBranchOrTagFails() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) .option(SparkReadOptions.BRANCH, "branch") .load(tableLocation) .show()) .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Can specify only one of snapshot-id"); + .hasMessageStartingWith("Cannot time travel in branch"); assertThatThrownBy( () -> spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) - .option(SparkReadOptions.TAG, "tag") + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) + .option(SparkReadOptions.VERSION_AS_OF, "tag") .load(tableLocation) .show()) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Can specify only one of snapshot-id"); + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot specify both version and timestamp"); } @TestTemplate @@ -565,7 +584,11 @@ public void testSnapshotSelectionByTagWithSchemaChange() { expectedRecords.addAll(firstBatchRecords); Dataset tagSnapshotResult = - spark.read().format("iceberg").option("tag", "tag").load(tableLocation); + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VERSION_AS_OF, "tag") + .load(tableLocation); List tagSnapshotRecords = tagSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); assertThat(tagSnapshotRecords) @@ -577,7 +600,11 @@ public void testSnapshotSelectionByTagWithSchemaChange() { // The data should have the deleted column as it was captured in an earlier snapshot. Dataset deletedColumnTagSnapshotResult = - spark.read().format("iceberg").option("tag", "tag").load(tableLocation); + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VERSION_AS_OF, "tag") + .load(tableLocation); List deletedColumnTagSnapshotRecords = deletedColumnTagSnapshotResult .orderBy("id") diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java index 1f266380cdc1..538a89a0bc3b 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java @@ -59,7 +59,7 @@ public Table loadTable(Identifier ident) throws NoSuchTableException { table = TestTables.load(tableIdentifier.name()); } - return new SparkTable(table, false); + return new SparkTable(table); } public static void clearTables() { diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java index f16db1972e7b..43b282c7e5d4 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java @@ -98,49 +98,36 @@ public static void beforeClass() { @TestTemplate public void testSparkSessionCatalogWithExpirationEnabled() { - SparkSessionCatalog sparkCatalog = sparkSessionCatalog(); - assertThat(sparkCatalog) - .extracting("icebergCatalog") - .extracting("cacheEnabled") - .isEqualTo(true); - - assertThat(sparkCatalog) - .extracting("icebergCatalog") - .extracting("icebergCatalog") + SparkSessionCatalog sparkSessionCatalog = sparkSessionCatalog(); + Catalog icebergCatalog = sparkSessionCatalog.icebergCatalog(); + + assertThat(icebergCatalog) .isInstanceOfSatisfying( - Catalog.class, - icebergCatalog -> { - assertThat(icebergCatalog) - .isExactlyInstanceOf(CachingCatalog.class) - .extracting("expirationIntervalMillis") - .isEqualTo(3000L); + CachingCatalog.class, + cachingCatalog -> { + assertThat(cachingCatalog).extracting("expirationIntervalMillis").isEqualTo(3000L); }); } @TestTemplate public void testCacheEnabledAndExpirationDisabled() { SparkCatalog sparkCatalog = getSparkCatalog("expiration_disabled"); - assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(true); + Catalog icebergCatalog = sparkCatalog.icebergCatalog(); - assertThat(sparkCatalog) - .extracting("icebergCatalog") + assertThat(icebergCatalog) .isInstanceOfSatisfying( CachingCatalog.class, - icebergCatalog -> { - assertThat(icebergCatalog).extracting("expirationIntervalMillis").isEqualTo(-1L); + cachingCatalog -> { + assertThat(cachingCatalog).extracting("expirationIntervalMillis").isEqualTo(-1L); }); } @TestTemplate public void testCacheDisabledImplicitly() { SparkCatalog sparkCatalog = getSparkCatalog("cache_disabled_implicitly"); - assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(false); + Catalog icebergCatalog = sparkCatalog.icebergCatalog(); - assertThat(sparkCatalog) - .extracting("icebergCatalog") - .isInstanceOfSatisfying( - Catalog.class, - icebergCatalog -> assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class)); + assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class); } private SparkSessionCatalog sparkSessionCatalog() { diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 94547c2cf8fb..465e6a5abf3d 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -685,7 +685,7 @@ public void testCommitUnknownException() { Table spyTable = spy(table); when(spyTable.newAppend()).thenReturn(spyAppend); - SparkTable sparkTable = new SparkTable(spyTable, false); + SparkTable sparkTable = new SparkTable(spyTable); String manualTableName = "unknown_exception"; ManualSource.setTable(manualTableName, sparkTable); diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index d9968c669719..fd133de4baa8 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -44,6 +44,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.spark.functions.BucketFunction; @@ -1026,6 +1027,7 @@ public void testPartitionedOr() throws Exception { @TestTemplate public void testBatchQueryScanDescription() throws Exception { createPartitionedTable(spark, tableName, "data"); + Table table = Spark3Util.loadIcebergTable(spark, tableName); SparkScanBuilder builder = scanBuilder(); withSQLConf( @@ -1040,6 +1042,9 @@ public void testBatchQueryScanDescription() throws Exception { assertThat(description).contains("IcebergScan"); assertThat(description).contains(tableName); + assertThat(description).contains("schemaId=" + table.schema().schemaId()); + assertThat(description).contains("snapshotId=" + table.currentSnapshot().snapshotId()); + assertThat(description).contains("branch=null"); assertThat(description).contains("filters=id = 1, id > 0"); assertThat(description).contains("groupedBy=data"); }); @@ -1048,6 +1053,7 @@ public void testBatchQueryScanDescription() throws Exception { @TestTemplate public void testCopyOnWriteScanDescription() throws Exception { createPartitionedTable(spark, tableName, "data"); + Table table = Spark3Util.loadIcebergTable(spark, tableName); SparkScanBuilder builder = scanBuilder(); withSQLConf( @@ -1062,17 +1068,51 @@ public void testCopyOnWriteScanDescription() throws Exception { assertThat(description).contains("IcebergCopyOnWriteScan"); assertThat(description).contains(tableName); + assertThat(description).contains("schemaId=" + table.schema().schemaId()); + assertThat(description).contains("snapshotId=" + table.currentSnapshot().snapshotId()); + assertThat(description).contains("branch=null"); assertThat(description).contains("filters=id = 2, id < 10"); assertThat(description).contains("groupedBy=data"); }); } - private SparkScanBuilder scanBuilder() throws Exception { + @TestTemplate + public void testIncrementalScanDescription() throws Exception { + createPartitionedTable(spark, tableName, "data"); Table table = Spark3Util.loadIcebergTable(spark, tableName); - CaseInsensitiveStringMap options = - new CaseInsensitiveStringMap(ImmutableMap.of("path", tableName)); + long startSnapshotId = table.currentSnapshot().snapshotId(); + + // add a second snapshot to use as the end + sql("INSERT INTO %s VALUES (1, CAST('2021-01-01 00:00:00' AS TIMESTAMP), 'b')", tableName); + table.refresh(); + long endSnapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder builder = + scanBuilder( + ImmutableMap.of( + SparkReadOptions.START_SNAPSHOT_ID, String.valueOf(startSnapshotId), + SparkReadOptions.END_SNAPSHOT_ID, String.valueOf(endSnapshotId))); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.PRESERVE_DATA_GROUPING, "true"), + () -> { + Scan scan = builder.build(); + String description = scan.description(); - return new SparkScanBuilder(spark, table, options); + assertThat(description).contains("IcebergIncrementalScan"); + assertThat(description).contains(tableName); + assertThat(description).contains("startSnapshotId=" + startSnapshotId); + assertThat(description).contains("endSnapshotId=" + endSnapshotId); + }); + } + + private SparkScanBuilder scanBuilder() throws Exception { + return scanBuilder(ImmutableMap.of()); + } + + private SparkScanBuilder scanBuilder(Map extraOptions) throws Exception { + Table table = Spark3Util.loadIcebergTable(spark, tableName); + return new SparkScanBuilder(spark, table, new CaseInsensitiveStringMap(extraOptions)); } private void pushFilters(ScanBuilder scan, Predicate... predicates) { diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java index 2389bcc17387..536d568003cf 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java @@ -86,7 +86,7 @@ public void testDeleteFromTableAtSnapshot() throws NoSuchTableException { String prefix = "snapshot_id_"; assertThatThrownBy(() -> sql("DELETE FROM %s.%s WHERE id < 4", tableName, prefix + snapshotId)) .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot delete from table at a specific snapshot"); + .hasMessageStartingWith("Cannot modify table with time travel"); } @TestTemplate diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index b6faaebe0416..6270296b01b0 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -38,7 +38,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.events.Listeners; import org.apache.iceberg.events.ScanEvent; -import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.hive.HiveCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -47,6 +46,7 @@ import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.types.Types; +import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.junit.jupiter.api.AfterEach; @@ -241,7 +241,7 @@ public void testSnapshotInTableName() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) + .option(SparkReadOptions.VERSION_AS_OF, snapshotId) .load(tableName) .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); @@ -253,6 +253,8 @@ public void testTimestampInTableName() { // get a timestamp just after the last write and get the current row set as expected long snapshotTs = validationCatalog.loadTable(tableIdent).currentSnapshot().timestampMillis(); long timestamp = waitUntilAfter(snapshotTs + 2); + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + String formattedDate = sdf.format(new Date(timestamp)); List expected = sql("SELECT * FROM %s ORDER by id", tableName); // create a second snapshot @@ -268,7 +270,7 @@ public void testTimestampInTableName() { spark .read() .format("iceberg") - .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedDate) .load(tableName) .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); @@ -338,7 +340,7 @@ public void testTagReference() { spark .read() .format("iceberg") - .option(SparkReadOptions.TAG, "test_tag") + .option(SparkReadOptions.VERSION_AS_OF, "test_tag") .load(tableName) .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); @@ -475,7 +477,7 @@ public void readAndWriteWithBranchAfterSchemaChange() { public void testUnknownReferenceAsOf() { assertThatThrownBy(() -> sql("SELECT * FROM %s VERSION AS OF 'test_unknown'", tableName)) .hasMessageContaining("Cannot find matching snapshot ID or reference name for version") - .isInstanceOf(ValidationException.class); + .isInstanceOf(IllegalArgumentException.class); } @TestTemplate @@ -548,7 +550,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { tableName, snapshotPrefix + snapshotId, snapshotId); }) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + .hasMessage("Can't time travel using selector and Spark time travel spec at the same time"); // using snapshot in table identifier and TIMESTAMP AS OF assertThatThrownBy( @@ -558,7 +560,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { tableName, timestampPrefix + timestamp, snapshotId); }) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + .hasMessage("Can't time travel using selector and Spark time travel spec at the same time"); // using timestamp in table identifier and VERSION AS OF assertThatThrownBy( @@ -568,7 +570,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { tableName, snapshotPrefix + snapshotId, timestamp); }) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + .hasMessage("Can't time travel using selector and Spark time travel spec at the same time"); // using timestamp in table identifier and TIMESTAMP AS OF assertThatThrownBy( @@ -578,7 +580,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { tableName, timestampPrefix + timestamp, timestamp); }) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + .hasMessage("Can't time travel using selector and Spark time travel spec at the same time"); } @TestTemplate @@ -593,12 +595,12 @@ public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { assertThatThrownBy( () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + .hasMessage("Can't time travel in branch"); // using branch_b1 in the table identifier and TIMESTAMP AS OF assertThatThrownBy(() -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + .hasMessage("Can't time travel in branch"); } @TestTemplate @@ -608,6 +610,8 @@ public void testSpecifySnapshotAndTimestamp() { // get a timestamp just after the last write long timestamp = validationCatalog.loadTable(tableIdent).currentSnapshot().timestampMillis() + 2; + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); + String formattedTimestamp = sdf.format(new Date(timestamp)); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); @@ -617,16 +621,13 @@ public void testSpecifySnapshotAndTimestamp() { spark .read() .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) - .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .option(SparkReadOptions.VERSION_AS_OF, snapshotId) + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedTimestamp) .load(tableName) .collectAsList(); }) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith( - String.format( - "Can specify only one of snapshot-id (%s), as-of-timestamp (%s)", - snapshotId, timestamp)); + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot specify both version and timestamp when time travelling"); } @TestTemplate diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java index 756f9c755ca0..65a624eac97c 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java @@ -101,7 +101,7 @@ public void testInsertAppendAtSnapshot() { () -> sql("INSERT INTO %s.%s VALUES (4, 'd'), (5, 'e')", tableName, prefix + snapshotId)) .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot write to table at a specific snapshot"); + .hasMessageStartingWith("Cannot write to table with time travel"); } @TestTemplate @@ -116,7 +116,7 @@ public void testInsertOverwriteAtSnapshot() { "INSERT OVERWRITE %s.%s VALUES (4, 'd'), (5, 'e')", tableName, prefix + snapshotId)) .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot write to table at a specific snapshot"); + .hasMessageStartingWith("Cannot write to table with time travel"); } @TestTemplate